| <!DOCTYPE HTML> |
| <html lang="en"> |
| <head> |
| <!-- Generated by javadoc (17) --> |
| <title>Source code</title> |
| <meta name="viewport" content="width=device-width, initial-scale=1"> |
| <meta name="description" content="source: package: org.apache.hadoop.hbase.test, class: IntegrationTestLoadCommonCrawl, class: OneFilePerMapperSFIF"> |
| <meta name="generator" content="javadoc/SourceToHTMLConverter"> |
| <link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style"> |
| </head> |
| <body class="source-page"> |
| <main role="main"> |
| <div class="source-container"> |
| <pre><span class="source-line-no">001</span><span id="line-1">/*</span> |
| <span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span> |
| <span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span> |
| <span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span> |
| <span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span> |
| <span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span> |
| <span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span> |
| <span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span> |
| <span class="source-line-no">009</span><span id="line-9"> *</span> |
| <span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span> |
| <span class="source-line-no">011</span><span id="line-11"> *</span> |
| <span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span> |
| <span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span> |
| <span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span> |
| <span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span> |
| <span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span> |
| <span class="source-line-no">017</span><span id="line-17"> */</span> |
| <span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.test;</span> |
| <span class="source-line-no">019</span><span id="line-19"></span> |
| <span class="source-line-no">020</span><span id="line-20">import static org.apache.hadoop.hbase.util.FutureUtils.addListener;</span> |
| <span class="source-line-no">021</span><span id="line-21"></span> |
| <span class="source-line-no">022</span><span id="line-22">import java.io.BufferedReader;</span> |
| <span class="source-line-no">023</span><span id="line-23">import java.io.ByteArrayOutputStream;</span> |
| <span class="source-line-no">024</span><span id="line-24">import java.io.DataInput;</span> |
| <span class="source-line-no">025</span><span id="line-25">import java.io.DataOutput;</span> |
| <span class="source-line-no">026</span><span id="line-26">import java.io.IOException;</span> |
| <span class="source-line-no">027</span><span id="line-27">import java.io.InputStreamReader;</span> |
| <span class="source-line-no">028</span><span id="line-28">import java.net.URI;</span> |
| <span class="source-line-no">029</span><span id="line-29">import java.net.URISyntaxException;</span> |
| <span class="source-line-no">030</span><span id="line-30">import java.nio.charset.StandardCharsets;</span> |
| <span class="source-line-no">031</span><span id="line-31">import java.util.ArrayList;</span> |
| <span class="source-line-no">032</span><span id="line-32">import java.util.Collection;</span> |
| <span class="source-line-no">033</span><span id="line-33">import java.util.HashSet;</span> |
| <span class="source-line-no">034</span><span id="line-34">import java.util.List;</span> |
| <span class="source-line-no">035</span><span id="line-35">import java.util.Set;</span> |
| <span class="source-line-no">036</span><span id="line-36">import java.util.concurrent.CompletableFuture;</span> |
| <span class="source-line-no">037</span><span id="line-37">import java.util.concurrent.ExecutionException;</span> |
| <span class="source-line-no">038</span><span id="line-38">import java.util.concurrent.ExecutorService;</span> |
| <span class="source-line-no">039</span><span id="line-39">import java.util.concurrent.Executors;</span> |
| <span class="source-line-no">040</span><span id="line-40">import java.util.concurrent.TimeUnit;</span> |
| <span class="source-line-no">041</span><span id="line-41">import java.util.concurrent.atomic.AtomicLong;</span> |
| <span class="source-line-no">042</span><span id="line-42">import java.util.regex.Matcher;</span> |
| <span class="source-line-no">043</span><span id="line-43">import java.util.regex.Pattern;</span> |
| <span class="source-line-no">044</span><span id="line-44">import java.util.zip.GZIPInputStream;</span> |
| <span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.conf.Configuration;</span> |
| <span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.conf.Configured;</span> |
| <span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.fs.FSDataInputStream;</span> |
| <span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.fs.FileSystem;</span> |
| <span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.fs.Path;</span> |
| <span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.Cell;</span> |
| <span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.HBaseConfiguration;</span> |
| <span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.HBaseTestingUtil;</span> |
| <span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.HConstants;</span> |
| <span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.IntegrationTestBase;</span> |
| <span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.IntegrationTestingUtility;</span> |
| <span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.MasterNotRunningException;</span> |
| <span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.TableName;</span> |
| <span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hbase.client.Admin;</span> |
| <span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hbase.client.AsyncConnection;</span> |
| <span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hbase.client.AsyncTable;</span> |
| <span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;</span> |
| <span class="source-line-no">062</span><span id="line-62">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;</span> |
| <span class="source-line-no">063</span><span id="line-63">import org.apache.hadoop.hbase.client.Connection;</span> |
| <span class="source-line-no">064</span><span id="line-64">import org.apache.hadoop.hbase.client.ConnectionFactory;</span> |
| <span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.hbase.client.Get;</span> |
| <span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.hbase.client.Increment;</span> |
| <span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.hbase.client.Put;</span> |
| <span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.hbase.client.Result;</span> |
| <span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.client.ScanResultConsumer;</span> |
| <span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.client.Table;</span> |
| <span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.client.TableDescriptor;</span> |
| <span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.client.TableDescriptorBuilder;</span> |
| <span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;</span> |
| <span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;</span> |
| <span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.regionserver.BloomType;</span> |
| <span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.test.util.CRC64;</span> |
| <span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.test.util.warc.WARCInputFormat;</span> |
| <span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.test.util.warc.WARCRecord;</span> |
| <span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.test.util.warc.WARCWritable;</span> |
| <span class="source-line-no">080</span><span id="line-80">import org.apache.hadoop.hbase.util.Bytes;</span> |
| <span class="source-line-no">081</span><span id="line-81">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span> |
| <span class="source-line-no">082</span><span id="line-82">import org.apache.hadoop.hbase.util.RegionSplitter;</span> |
| <span class="source-line-no">083</span><span id="line-83">import org.apache.hadoop.io.BytesWritable;</span> |
| <span class="source-line-no">084</span><span id="line-84">import org.apache.hadoop.io.LongWritable;</span> |
| <span class="source-line-no">085</span><span id="line-85">import org.apache.hadoop.io.NullWritable;</span> |
| <span class="source-line-no">086</span><span id="line-86">import org.apache.hadoop.io.SequenceFile.CompressionType;</span> |
| <span class="source-line-no">087</span><span id="line-87">import org.apache.hadoop.io.Writable;</span> |
| <span class="source-line-no">088</span><span id="line-88">import org.apache.hadoop.mapreduce.Counters;</span> |
| <span class="source-line-no">089</span><span id="line-89">import org.apache.hadoop.mapreduce.Job;</span> |
| <span class="source-line-no">090</span><span id="line-90">import org.apache.hadoop.mapreduce.JobContext;</span> |
| <span class="source-line-no">091</span><span id="line-91">import org.apache.hadoop.mapreduce.Mapper;</span> |
| <span class="source-line-no">092</span><span id="line-92">import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;</span> |
| <span class="source-line-no">093</span><span id="line-93">import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;</span> |
| <span class="source-line-no">094</span><span id="line-94">import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;</span> |
| <span class="source-line-no">095</span><span id="line-95">import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;</span> |
| <span class="source-line-no">096</span><span id="line-96">import org.apache.hadoop.util.Tool;</span> |
| <span class="source-line-no">097</span><span id="line-97">import org.apache.hadoop.util.ToolRunner;</span> |
| <span class="source-line-no">098</span><span id="line-98">import org.slf4j.Logger;</span> |
| <span class="source-line-no">099</span><span id="line-99">import org.slf4j.LoggerFactory;</span> |
| <span class="source-line-no">100</span><span id="line-100"></span> |
| <span class="source-line-no">101</span><span id="line-101">import org.apache.hbase.thirdparty.com.google.common.base.Splitter;</span> |
| <span class="source-line-no">102</span><span id="line-102">import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;</span> |
| <span class="source-line-no">103</span><span id="line-103"></span> |
| <span class="source-line-no">104</span><span id="line-104">/**</span> |
| <span class="source-line-no">105</span><span id="line-105"> * This integration test loads successful resource retrieval records from the Common Crawl</span> |
| <span class="source-line-no">106</span><span id="line-106"> * (https://commoncrawl.org/) public dataset into an HBase table and writes records that can be used</span> |
| <span class="source-line-no">107</span><span id="line-107"> * to later verify the presence and integrity of those records.</span> |
| <span class="source-line-no">108</span><span id="line-108"> * <p></span> |
| <span class="source-line-no">109</span><span id="line-109"> * Run like: <blockquote> ./bin/hbase org.apache.hadoop.hbase.test.IntegrationTestLoadCommonCrawl</span> |
| <span class="source-line-no">110</span><span id="line-110"> * \<br></span> |
| <span class="source-line-no">111</span><span id="line-111"> * &nbsp;&nbsp; -Dfs.s3n.awsAccessKeyId=&lt;AWS access key&gt; \<br></span> |
| <span class="source-line-no">112</span><span id="line-112"> * &nbsp;&nbsp; -Dfs.s3n.awsSecretAccessKey=&lt;AWS secret key&gt; \<br></span> |
| <span class="source-line-no">113</span><span id="line-113"> * &nbsp;&nbsp; /path/to/test-CC-MAIN-2021-10-warc.paths.gz \<br></span> |
| <span class="source-line-no">114</span><span id="line-114"> * &nbsp;&nbsp; /path/to/tmp/warc-loader-output </blockquote></span> |
| <span class="source-line-no">115</span><span id="line-115"> * <p></span> |
| <span class="source-line-no">116</span><span id="line-116"> * Access to the Common Crawl dataset in S3 is made available to anyone by Amazon AWS, but Hadoop's</span> |
| <span class="source-line-no">117</span><span id="line-117"> * S3N filesystem still requires valid access credentials to initialize.</span> |
| <span class="source-line-no">118</span><span id="line-118"> * <p></span> |
| <span class="source-line-no">119</span><span id="line-119"> * The input path can either specify a directory or a file. The file may optionally be compressed</span> |
| <span class="source-line-no">120</span><span id="line-120"> * with gzip. If a directory, the loader expects the directory to contain one or more WARC files</span> |
| <span class="source-line-no">121</span><span id="line-121"> * from the Common Crawl dataset. If a file, the loader expects a list of Hadoop S3N URIs which</span> |
| <span class="source-line-no">122</span><span id="line-122"> * point to S3 locations for one or more WARC files from the Common Crawl dataset, one URI per line.</span> |
| <span class="source-line-no">123</span><span id="line-123"> * Lines should be terminated with the UNIX line terminator.</span> |
| <span class="source-line-no">124</span><span id="line-124"> * <p></span> |
| <span class="source-line-no">125</span><span id="line-125"> * Included in hbase-it/src/test/resources/CC-MAIN-2021-10-warc.paths.gz is a list of all WARC files</span> |
| <span class="source-line-no">126</span><span id="line-126"> * comprising the Q1 2021 crawl archive. There are 64,000 WARC files in this data set, each</span> |
| <span class="source-line-no">127</span><span id="line-127"> * containing ~1GB of gzipped data. The WARC files contain several record types, such as metadata,</span> |
| <span class="source-line-no">128</span><span id="line-128"> * request, and response, but we only load the response record types. If the HBase table schema does</span> |
| <span class="source-line-no">129</span><span id="line-129"> * not specify compression (by default) there is roughly a 10x expansion. Loading the full crawl</span> |
| <span class="source-line-no">130</span><span id="line-130"> * archive results in a table approximately 640 TB in size.</span> |
| <span class="source-line-no">131</span><span id="line-131"> * <p></span> |
| <span class="source-line-no">132</span><span id="line-132"> * The loader can optionally drive read load during ingest by incrementing counters for each URL</span> |
| <span class="source-line-no">133</span><span id="line-133"> * discovered in content. Add <tt>-DIntegrationTestLoadCommonCrawl.increments=true</tt> to the</span> |
| <span class="source-line-no">134</span><span id="line-134"> * command line to enable.</span> |
| <span class="source-line-no">135</span><span id="line-135"> * <p></span> |
| <span class="source-line-no">136</span><span id="line-136"> * You can also split the Loader and Verify stages:</span> |
| <span class="source-line-no">137</span><span id="line-137"> * <p></span> |
| <span class="source-line-no">138</span><span id="line-138"> * Load with: <blockquote> ./bin/hbase</span> |
| <span class="source-line-no">139</span><span id="line-139"> * 'org.apache.hadoop.hbase.test.IntegrationTestLoadCommonCrawl$Loader' \<br></span> |
| <span class="source-line-no">140</span><span id="line-140"> * &nbsp;&nbsp; -files /path/to/hadoop-aws.jar \<br></span> |
| <span class="source-line-no">141</span><span id="line-141"> * &nbsp;&nbsp; -Dfs.s3n.awsAccessKeyId=&lt;AWS access key&gt; \<br></span> |
| <span class="source-line-no">142</span><span id="line-142"> * &nbsp;&nbsp; -Dfs.s3n.awsSecretAccessKey=&lt;AWS secret key&gt; \<br></span> |
| <span class="source-line-no">143</span><span id="line-143"> * &nbsp;&nbsp; /path/to/test-CC-MAIN-2021-10-warc.paths.gz \<br></span> |
| <span class="source-line-no">144</span><span id="line-144"> * &nbsp;&nbsp; /path/to/tmp/warc-loader-output </blockquote></span> |
| <span class="source-line-no">145</span><span id="line-145"> * <p></span> |
| <span class="source-line-no">146</span><span id="line-146"> * Note: The hadoop-aws jar will be needed at runtime to instantiate the S3N filesystem. Use the</span> |
| <span class="source-line-no">147</span><span id="line-147"> * <tt>-files</tt> ToolRunner argument to add it.</span> |
| <span class="source-line-no">148</span><span id="line-148"> * <p></span> |
| <span class="source-line-no">149</span><span id="line-149"> * Verify with: <blockquote> ./bin/hbase</span> |
| <span class="source-line-no">150</span><span id="line-150"> * 'org.apache.hadoop.hbase.test.IntegrationTestLoadCommonCrawl$Verify' \<br></span> |
| <span class="source-line-no">151</span><span id="line-151"> * &nbsp;&nbsp; /path/to/tmp/warc-loader-output </blockquote></span> |
| <span class="source-line-no">152</span><span id="line-152"> * <p></span> |
| <span class="source-line-no">153</span><span id="line-153"> */</span> |
| <span class="source-line-no">154</span><span id="line-154">public class IntegrationTestLoadCommonCrawl extends IntegrationTestBase {</span> |
| <span class="source-line-no">155</span><span id="line-155"></span> |
| <span class="source-line-no">156</span><span id="line-156"> private static final Logger LOG = LoggerFactory.getLogger(IntegrationTestLoadCommonCrawl.class);</span> |
| <span class="source-line-no">157</span><span id="line-157"></span> |
| <span class="source-line-no">158</span><span id="line-158"> static final String TABLE_NAME_KEY = "IntegrationTestLoadCommonCrawl.table";</span> |
| <span class="source-line-no">159</span><span id="line-159"> static final String DEFAULT_TABLE_NAME = "IntegrationTestLoadCommonCrawl";</span> |
| <span class="source-line-no">160</span><span id="line-160"></span> |
| <span class="source-line-no">161</span><span id="line-161"> static final String INCREMENTS_NAME_KEY = "IntegrationTestLoadCommonCrawl.increments";</span> |
| <span class="source-line-no">162</span><span id="line-162"> static final boolean DEFAULT_INCREMENTS = false;</span> |
| <span class="source-line-no">163</span><span id="line-163"></span> |
| <span class="source-line-no">164</span><span id="line-164"> static final int MAX_INFLIGHT = 1000;</span> |
| <span class="source-line-no">165</span><span id="line-165"> static final int INFLIGHT_PAUSE_MS = 100;</span> |
| <span class="source-line-no">166</span><span id="line-166"></span> |
| <span class="source-line-no">167</span><span id="line-167"> static final byte[] CONTENT_FAMILY_NAME = Bytes.toBytes("c");</span> |
| <span class="source-line-no">168</span><span id="line-168"> static final byte[] INFO_FAMILY_NAME = Bytes.toBytes("i");</span> |
| <span class="source-line-no">169</span><span id="line-169"> static final byte[] URL_FAMILY_NAME = Bytes.toBytes("u");</span> |
| <span class="source-line-no">170</span><span id="line-170"> static final byte[] SEP = Bytes.toBytes(":");</span> |
| <span class="source-line-no">171</span><span id="line-171"> static final byte[] CONTENT_QUALIFIER = HConstants.EMPTY_BYTE_ARRAY;</span> |
| <span class="source-line-no">172</span><span id="line-172"> static final byte[] CONTENT_LENGTH_QUALIFIER = Bytes.toBytes("l");</span> |
| <span class="source-line-no">173</span><span id="line-173"> static final byte[] CONTENT_TYPE_QUALIFIER = Bytes.toBytes("t");</span> |
| <span class="source-line-no">174</span><span id="line-174"> static final byte[] CRC_QUALIFIER = Bytes.toBytes("c");</span> |
| <span class="source-line-no">175</span><span id="line-175"> static final byte[] DATE_QUALIFIER = Bytes.toBytes("d");</span> |
| <span class="source-line-no">176</span><span id="line-176"> static final byte[] IP_ADDRESS_QUALIFIER = Bytes.toBytes("a");</span> |
| <span class="source-line-no">177</span><span id="line-177"> static final byte[] TARGET_URI_QUALIFIER = Bytes.toBytes("u");</span> |
| <span class="source-line-no">178</span><span id="line-178"> static final byte[] REF_QUALIFIER = Bytes.toBytes("ref");</span> |
| <span class="source-line-no">179</span><span id="line-179"></span> |
| <span class="source-line-no">180</span><span id="line-180"> public static enum Counts {</span> |
| <span class="source-line-no">181</span><span id="line-181"> REFERENCED,</span> |
| <span class="source-line-no">182</span><span id="line-182"> UNREFERENCED,</span> |
| <span class="source-line-no">183</span><span id="line-183"> CORRUPT,</span> |
| <span class="source-line-no">184</span><span id="line-184"> RPC_BYTES_WRITTEN,</span> |
| <span class="source-line-no">185</span><span id="line-185"> RPC_TIME_MS,</span> |
| <span class="source-line-no">186</span><span id="line-186"> }</span> |
| <span class="source-line-no">187</span><span id="line-187"></span> |
| <span class="source-line-no">188</span><span id="line-188"> protected Path warcFileInputDir = null;</span> |
| <span class="source-line-no">189</span><span id="line-189"> protected Path outputDir = null;</span> |
| <span class="source-line-no">190</span><span id="line-190"> protected String[] args;</span> |
| <span class="source-line-no">191</span><span id="line-191"></span> |
| <span class="source-line-no">192</span><span id="line-192"> protected int runLoader(final Path warcFileInputDir, final Path outputDir) throws Exception {</span> |
| <span class="source-line-no">193</span><span id="line-193"> Loader loader = new Loader();</span> |
| <span class="source-line-no">194</span><span id="line-194"> loader.setConf(conf);</span> |
| <span class="source-line-no">195</span><span id="line-195"> return loader.run(warcFileInputDir, outputDir);</span> |
| <span class="source-line-no">196</span><span id="line-196"> }</span> |
| <span class="source-line-no">197</span><span id="line-197"></span> |
| <span class="source-line-no">198</span><span id="line-198"> protected int runVerify(final Path inputDir) throws Exception {</span> |
| <span class="source-line-no">199</span><span id="line-199"> Verify verify = new Verify();</span> |
| <span class="source-line-no">200</span><span id="line-200"> verify.setConf(conf);</span> |
| <span class="source-line-no">201</span><span id="line-201"> return verify.run(inputDir);</span> |
| <span class="source-line-no">202</span><span id="line-202"> }</span> |
| <span class="source-line-no">203</span><span id="line-203"></span> |
| <span class="source-line-no">204</span><span id="line-204"> @Override</span> |
| <span class="source-line-no">205</span><span id="line-205"> public int run(String[] args) {</span> |
| <span class="source-line-no">206</span><span id="line-206"> if (args.length > 0) {</span> |
| <span class="source-line-no">207</span><span id="line-207"> warcFileInputDir = new Path(args[0]);</span> |
| <span class="source-line-no">208</span><span id="line-208"> if (args.length > 1) {</span> |
| <span class="source-line-no">209</span><span id="line-209"> outputDir = new Path(args[1]);</span> |
| <span class="source-line-no">210</span><span id="line-210"> }</span> |
| <span class="source-line-no">211</span><span id="line-211"> }</span> |
| <span class="source-line-no">212</span><span id="line-212"> try {</span> |
| <span class="source-line-no">213</span><span id="line-213"> if (warcFileInputDir == null) {</span> |
| <span class="source-line-no">214</span><span id="line-214"> throw new IllegalArgumentException("WARC input file or directory not specified");</span> |
| <span class="source-line-no">215</span><span id="line-215"> }</span> |
| <span class="source-line-no">216</span><span id="line-216"> if (outputDir == null) {</span> |
| <span class="source-line-no">217</span><span id="line-217"> throw new IllegalArgumentException("Output directory not specified");</span> |
| <span class="source-line-no">218</span><span id="line-218"> }</span> |
| <span class="source-line-no">219</span><span id="line-219"> int res = runLoader(warcFileInputDir, outputDir);</span> |
| <span class="source-line-no">220</span><span id="line-220"> if (res != 0) {</span> |
| <span class="source-line-no">221</span><span id="line-221"> LOG.error("Loader failed");</span> |
| <span class="source-line-no">222</span><span id="line-222"> return -1;</span> |
| <span class="source-line-no">223</span><span id="line-223"> }</span> |
| <span class="source-line-no">224</span><span id="line-224"> return runVerify(outputDir);</span> |
| <span class="source-line-no">225</span><span id="line-225"> } catch (Exception e) {</span> |
| <span class="source-line-no">226</span><span id="line-226"> LOG.error("Tool failed with exception", e);</span> |
| <span class="source-line-no">227</span><span id="line-227"> return -1;</span> |
| <span class="source-line-no">228</span><span id="line-228"> }</span> |
| <span class="source-line-no">229</span><span id="line-229"> }</span> |
| <span class="source-line-no">230</span><span id="line-230"></span> |
| <span class="source-line-no">231</span><span id="line-231"> @Override</span> |
| <span class="source-line-no">232</span><span id="line-232"> protected void processOptions(final CommandLine cmd) {</span> |
| <span class="source-line-no">233</span><span id="line-233"> processBaseOptions(cmd);</span> |
| <span class="source-line-no">234</span><span id="line-234"> args = cmd.getArgs();</span> |
| <span class="source-line-no">235</span><span id="line-235"> }</span> |
| <span class="source-line-no">236</span><span id="line-236"></span> |
| <span class="source-line-no">237</span><span id="line-237"> @Override</span> |
| <span class="source-line-no">238</span><span id="line-238"> public void setUpCluster() throws Exception {</span> |
| <span class="source-line-no">239</span><span id="line-239"> util = getTestingUtil(getConf());</span> |
| <span class="source-line-no">240</span><span id="line-240"> boolean isDistributed = util.isDistributedCluster();</span> |
| <span class="source-line-no">241</span><span id="line-241"> util.initializeCluster(isDistributed ? 1 : 3);</span> |
| <span class="source-line-no">242</span><span id="line-242"> if (!isDistributed) {</span> |
| <span class="source-line-no">243</span><span id="line-243"> util.startMiniMapReduceCluster();</span> |
| <span class="source-line-no">244</span><span id="line-244"> }</span> |
| <span class="source-line-no">245</span><span id="line-245"> this.setConf(util.getConfiguration());</span> |
| <span class="source-line-no">246</span><span id="line-246"> }</span> |
| <span class="source-line-no">247</span><span id="line-247"></span> |
| <span class="source-line-no">248</span><span id="line-248"> @Override</span> |
| <span class="source-line-no">249</span><span id="line-249"> public void cleanUpCluster() throws Exception {</span> |
| <span class="source-line-no">250</span><span id="line-250"> super.cleanUpCluster();</span> |
| <span class="source-line-no">251</span><span id="line-251"> if (util.isDistributedCluster()) {</span> |
| <span class="source-line-no">252</span><span id="line-252"> util.shutdownMiniMapReduceCluster();</span> |
| <span class="source-line-no">253</span><span id="line-253"> }</span> |
| <span class="source-line-no">254</span><span id="line-254"> }</span> |
| <span class="source-line-no">255</span><span id="line-255"></span> |
| <span class="source-line-no">256</span><span id="line-256"> static TableName getTablename(final Configuration c) {</span> |
| <span class="source-line-no">257</span><span id="line-257"> return TableName.valueOf(c.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));</span> |
| <span class="source-line-no">258</span><span id="line-258"> }</span> |
| <span class="source-line-no">259</span><span id="line-259"></span> |
| <span class="source-line-no">260</span><span id="line-260"> @Override</span> |
| <span class="source-line-no">261</span><span id="line-261"> public TableName getTablename() {</span> |
| <span class="source-line-no">262</span><span id="line-262"> return getTablename(getConf());</span> |
| <span class="source-line-no">263</span><span id="line-263"> }</span> |
| <span class="source-line-no">264</span><span id="line-264"></span> |
| <span class="source-line-no">265</span><span id="line-265"> @Override</span> |
| <span class="source-line-no">266</span><span id="line-266"> protected Set<String> getColumnFamilies() {</span> |
| <span class="source-line-no">267</span><span id="line-267"> Set<String> families = new HashSet<>();</span> |
| <span class="source-line-no">268</span><span id="line-268"> families.add(Bytes.toString(CONTENT_FAMILY_NAME));</span> |
| <span class="source-line-no">269</span><span id="line-269"> families.add(Bytes.toString(INFO_FAMILY_NAME));</span> |
| <span class="source-line-no">270</span><span id="line-270"> families.add(Bytes.toString(URL_FAMILY_NAME));</span> |
| <span class="source-line-no">271</span><span id="line-271"> return families;</span> |
| <span class="source-line-no">272</span><span id="line-272"> }</span> |
| <span class="source-line-no">273</span><span id="line-273"></span> |
| <span class="source-line-no">274</span><span id="line-274"> @Override</span> |
| <span class="source-line-no">275</span><span id="line-275"> public int runTestFromCommandLine() throws Exception {</span> |
| <span class="source-line-no">276</span><span id="line-276"> return ToolRunner.run(getConf(), this, args);</span> |
| <span class="source-line-no">277</span><span id="line-277"> }</span> |
| <span class="source-line-no">278</span><span id="line-278"></span> |
| <span class="source-line-no">279</span><span id="line-279"> public static void main(String[] args) throws Exception {</span> |
| <span class="source-line-no">280</span><span id="line-280"> Configuration conf = HBaseConfiguration.create();</span> |
| <span class="source-line-no">281</span><span id="line-281"> IntegrationTestingUtility.setUseDistributedCluster(conf);</span> |
| <span class="source-line-no">282</span><span id="line-282"> int ret = ToolRunner.run(conf, new IntegrationTestLoadCommonCrawl(), args);</span> |
| <span class="source-line-no">283</span><span id="line-283"> System.exit(ret);</span> |
| <span class="source-line-no">284</span><span id="line-284"> }</span> |
| <span class="source-line-no">285</span><span id="line-285"></span> |
| <span class="source-line-no">286</span><span id="line-286"> public static class HBaseKeyWritable implements Writable {</span> |
| <span class="source-line-no">287</span><span id="line-287"></span> |
| <span class="source-line-no">288</span><span id="line-288"> private byte[] row;</span> |
| <span class="source-line-no">289</span><span id="line-289"> private int rowOffset;</span> |
| <span class="source-line-no">290</span><span id="line-290"> private int rowLength;</span> |
| <span class="source-line-no">291</span><span id="line-291"> private byte[] family;</span> |
| <span class="source-line-no">292</span><span id="line-292"> private int familyOffset;</span> |
| <span class="source-line-no">293</span><span id="line-293"> private int familyLength;</span> |
| <span class="source-line-no">294</span><span id="line-294"> private byte[] qualifier;</span> |
| <span class="source-line-no">295</span><span id="line-295"> private int qualifierOffset;</span> |
| <span class="source-line-no">296</span><span id="line-296"> private int qualifierLength;</span> |
| <span class="source-line-no">297</span><span id="line-297"> private long ts;</span> |
| <span class="source-line-no">298</span><span id="line-298"></span> |
| <span class="source-line-no">299</span><span id="line-299"> public HBaseKeyWritable() {</span> |
| <span class="source-line-no">300</span><span id="line-300"> }</span> |
| <span class="source-line-no">301</span><span id="line-301"></span> |
| <span class="source-line-no">302</span><span id="line-302"> public HBaseKeyWritable(byte[] row, int rowOffset, int rowLength, byte[] family,</span> |
| <span class="source-line-no">303</span><span id="line-303"> int familyOffset, int familyLength, byte[] qualifier, int qualifierOffset,</span> |
| <span class="source-line-no">304</span><span id="line-304"> int qualifierLength, long ts) {</span> |
| <span class="source-line-no">305</span><span id="line-305"> this.row = row;</span> |
| <span class="source-line-no">306</span><span id="line-306"> this.rowOffset = rowOffset;</span> |
| <span class="source-line-no">307</span><span id="line-307"> this.rowLength = rowLength;</span> |
| <span class="source-line-no">308</span><span id="line-308"> this.family = family;</span> |
| <span class="source-line-no">309</span><span id="line-309"> this.familyOffset = familyOffset;</span> |
| <span class="source-line-no">310</span><span id="line-310"> this.familyLength = familyLength;</span> |
| <span class="source-line-no">311</span><span id="line-311"> this.qualifier = qualifier;</span> |
| <span class="source-line-no">312</span><span id="line-312"> this.qualifierOffset = qualifierOffset;</span> |
| <span class="source-line-no">313</span><span id="line-313"> this.qualifierLength = qualifierLength;</span> |
| <span class="source-line-no">314</span><span id="line-314"> this.ts = ts;</span> |
| <span class="source-line-no">315</span><span id="line-315"> }</span> |
| <span class="source-line-no">316</span><span id="line-316"></span> |
| <span class="source-line-no">317</span><span id="line-317"> public HBaseKeyWritable(byte[] row, byte[] family, byte[] qualifier, long ts) {</span> |
| <span class="source-line-no">318</span><span id="line-318"> this(row, 0, row.length, family, 0, family.length, qualifier, 0,</span> |
| <span class="source-line-no">319</span><span id="line-319"> qualifier != null ? qualifier.length : 0, ts);</span> |
| <span class="source-line-no">320</span><span id="line-320"> }</span> |
| <span class="source-line-no">321</span><span id="line-321"></span> |
| <span class="source-line-no">322</span><span id="line-322"> public HBaseKeyWritable(byte[] row, byte[] family, byte[] qualifier) {</span> |
| <span class="source-line-no">323</span><span id="line-323"> this(row, family, qualifier, Long.MAX_VALUE);</span> |
| <span class="source-line-no">324</span><span id="line-324"> }</span> |
| <span class="source-line-no">325</span><span id="line-325"></span> |
| <span class="source-line-no">326</span><span id="line-326"> public HBaseKeyWritable(byte[] row, byte[] family, long ts) {</span> |
| <span class="source-line-no">327</span><span id="line-327"> this(row, family, HConstants.EMPTY_BYTE_ARRAY, ts);</span> |
| <span class="source-line-no">328</span><span id="line-328"> }</span> |
| <span class="source-line-no">329</span><span id="line-329"></span> |
| <span class="source-line-no">330</span><span id="line-330"> public HBaseKeyWritable(byte[] row, byte[] family) {</span> |
| <span class="source-line-no">331</span><span id="line-331"> this(row, family, Long.MAX_VALUE);</span> |
| <span class="source-line-no">332</span><span id="line-332"> }</span> |
| <span class="source-line-no">333</span><span id="line-333"></span> |
| <span class="source-line-no">334</span><span id="line-334"> public HBaseKeyWritable(Cell cell) {</span> |
| <span class="source-line-no">335</span><span id="line-335"> this(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), cell.getFamilyArray(),</span> |
| <span class="source-line-no">336</span><span id="line-336"> cell.getFamilyOffset(), cell.getFamilyLength(), cell.getQualifierArray(),</span> |
| <span class="source-line-no">337</span><span id="line-337"> cell.getQualifierOffset(), cell.getQualifierLength(), cell.getTimestamp());</span> |
| <span class="source-line-no">338</span><span id="line-338"> }</span> |
| <span class="source-line-no">339</span><span id="line-339"></span> |
| <span class="source-line-no">340</span><span id="line-340"> @Override</span> |
| <span class="source-line-no">341</span><span id="line-341"> public void readFields(DataInput in) throws IOException {</span> |
| <span class="source-line-no">342</span><span id="line-342"> this.row = Bytes.toBytes(in.readUTF());</span> |
| <span class="source-line-no">343</span><span id="line-343"> this.rowOffset = 0;</span> |
| <span class="source-line-no">344</span><span id="line-344"> this.rowLength = row.length;</span> |
| <span class="source-line-no">345</span><span id="line-345"> this.family = Bytes.toBytes(in.readUTF());</span> |
| <span class="source-line-no">346</span><span id="line-346"> this.familyOffset = 0;</span> |
| <span class="source-line-no">347</span><span id="line-347"> this.familyLength = family.length;</span> |
| <span class="source-line-no">348</span><span id="line-348"> this.qualifier = Bytes.toBytes(in.readUTF());</span> |
| <span class="source-line-no">349</span><span id="line-349"> this.qualifierOffset = 0;</span> |
| <span class="source-line-no">350</span><span id="line-350"> this.qualifierLength = qualifier.length;</span> |
| <span class="source-line-no">351</span><span id="line-351"> this.ts = in.readLong();</span> |
| <span class="source-line-no">352</span><span id="line-352"> }</span> |
| <span class="source-line-no">353</span><span id="line-353"></span> |
| <span class="source-line-no">354</span><span id="line-354"> @Override</span> |
| <span class="source-line-no">355</span><span id="line-355"> public void write(DataOutput out) throws IOException {</span> |
| <span class="source-line-no">356</span><span id="line-356"> out.writeUTF(new String(row, rowOffset, rowLength, StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">357</span><span id="line-357"> out.writeUTF(new String(family, familyOffset, familyLength, StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">358</span><span id="line-358"> if (qualifier != null) {</span> |
| <span class="source-line-no">359</span><span id="line-359"> out.writeUTF(</span> |
| <span class="source-line-no">360</span><span id="line-360"> new String(qualifier, qualifierOffset, qualifierLength, StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">361</span><span id="line-361"> } else {</span> |
| <span class="source-line-no">362</span><span id="line-362"> out.writeUTF("");</span> |
| <span class="source-line-no">363</span><span id="line-363"> }</span> |
| <span class="source-line-no">364</span><span id="line-364"> out.writeLong(ts);</span> |
| <span class="source-line-no">365</span><span id="line-365"> }</span> |
| <span class="source-line-no">366</span><span id="line-366"></span> |
| <span class="source-line-no">367</span><span id="line-367"> public byte[] getRowArray() {</span> |
| <span class="source-line-no">368</span><span id="line-368"> return row;</span> |
| <span class="source-line-no">369</span><span id="line-369"> }</span> |
| <span class="source-line-no">370</span><span id="line-370"></span> |
| <span class="source-line-no">371</span><span id="line-371"> public void setRow(byte[] row) {</span> |
| <span class="source-line-no">372</span><span id="line-372"> this.row = row;</span> |
| <span class="source-line-no">373</span><span id="line-373"> }</span> |
| <span class="source-line-no">374</span><span id="line-374"></span> |
| <span class="source-line-no">375</span><span id="line-375"> public int getRowOffset() {</span> |
| <span class="source-line-no">376</span><span id="line-376"> return rowOffset;</span> |
| <span class="source-line-no">377</span><span id="line-377"> }</span> |
| <span class="source-line-no">378</span><span id="line-378"></span> |
| <span class="source-line-no">379</span><span id="line-379"> public void setRowOffset(int rowOffset) {</span> |
| <span class="source-line-no">380</span><span id="line-380"> this.rowOffset = rowOffset;</span> |
| <span class="source-line-no">381</span><span id="line-381"> }</span> |
| <span class="source-line-no">382</span><span id="line-382"></span> |
| <span class="source-line-no">383</span><span id="line-383"> public int getRowLength() {</span> |
| <span class="source-line-no">384</span><span id="line-384"> return rowLength;</span> |
| <span class="source-line-no">385</span><span id="line-385"> }</span> |
| <span class="source-line-no">386</span><span id="line-386"></span> |
| <span class="source-line-no">387</span><span id="line-387"> public void setRowLength(int rowLength) {</span> |
| <span class="source-line-no">388</span><span id="line-388"> this.rowLength = rowLength;</span> |
| <span class="source-line-no">389</span><span id="line-389"> }</span> |
| <span class="source-line-no">390</span><span id="line-390"></span> |
| <span class="source-line-no">391</span><span id="line-391"> public byte[] getFamilyArray() {</span> |
| <span class="source-line-no">392</span><span id="line-392"> return family;</span> |
| <span class="source-line-no">393</span><span id="line-393"> }</span> |
| <span class="source-line-no">394</span><span id="line-394"></span> |
| <span class="source-line-no">395</span><span id="line-395"> public void setFamily(byte[] family) {</span> |
| <span class="source-line-no">396</span><span id="line-396"> this.family = family;</span> |
| <span class="source-line-no">397</span><span id="line-397"> }</span> |
| <span class="source-line-no">398</span><span id="line-398"></span> |
| <span class="source-line-no">399</span><span id="line-399"> public int getFamilyOffset() {</span> |
| <span class="source-line-no">400</span><span id="line-400"> return familyOffset;</span> |
| <span class="source-line-no">401</span><span id="line-401"> }</span> |
| <span class="source-line-no">402</span><span id="line-402"></span> |
| <span class="source-line-no">403</span><span id="line-403"> public void setFamilyOffset(int familyOffset) {</span> |
| <span class="source-line-no">404</span><span id="line-404"> this.familyOffset = familyOffset;</span> |
| <span class="source-line-no">405</span><span id="line-405"> }</span> |
| <span class="source-line-no">406</span><span id="line-406"></span> |
| <span class="source-line-no">407</span><span id="line-407"> public int getFamilyLength() {</span> |
| <span class="source-line-no">408</span><span id="line-408"> return familyLength;</span> |
| <span class="source-line-no">409</span><span id="line-409"> }</span> |
| <span class="source-line-no">410</span><span id="line-410"></span> |
| <span class="source-line-no">411</span><span id="line-411"> public void setFamilyLength(int familyLength) {</span> |
| <span class="source-line-no">412</span><span id="line-412"> this.familyLength = familyLength;</span> |
| <span class="source-line-no">413</span><span id="line-413"> }</span> |
| <span class="source-line-no">414</span><span id="line-414"></span> |
| <span class="source-line-no">415</span><span id="line-415"> public byte[] getQualifierArray() {</span> |
| <span class="source-line-no">416</span><span id="line-416"> return qualifier;</span> |
| <span class="source-line-no">417</span><span id="line-417"> }</span> |
| <span class="source-line-no">418</span><span id="line-418"></span> |
| <span class="source-line-no">419</span><span id="line-419"> public void setQualifier(byte[] qualifier) {</span> |
| <span class="source-line-no">420</span><span id="line-420"> this.qualifier = qualifier;</span> |
| <span class="source-line-no">421</span><span id="line-421"> }</span> |
| <span class="source-line-no">422</span><span id="line-422"></span> |
| <span class="source-line-no">423</span><span id="line-423"> public int getQualifierOffset() {</span> |
| <span class="source-line-no">424</span><span id="line-424"> return qualifierOffset;</span> |
| <span class="source-line-no">425</span><span id="line-425"> }</span> |
| <span class="source-line-no">426</span><span id="line-426"></span> |
| <span class="source-line-no">427</span><span id="line-427"> public void setQualifierOffset(int qualifierOffset) {</span> |
| <span class="source-line-no">428</span><span id="line-428"> this.qualifierOffset = qualifierOffset;</span> |
| <span class="source-line-no">429</span><span id="line-429"> }</span> |
| <span class="source-line-no">430</span><span id="line-430"></span> |
| <span class="source-line-no">431</span><span id="line-431"> public int getQualifierLength() {</span> |
| <span class="source-line-no">432</span><span id="line-432"> return qualifierLength;</span> |
| <span class="source-line-no">433</span><span id="line-433"> }</span> |
| <span class="source-line-no">434</span><span id="line-434"></span> |
| <span class="source-line-no">435</span><span id="line-435"> public void setQualifierLength(int qualifierLength) {</span> |
| <span class="source-line-no">436</span><span id="line-436"> this.qualifierLength = qualifierLength;</span> |
| <span class="source-line-no">437</span><span id="line-437"> }</span> |
| <span class="source-line-no">438</span><span id="line-438"></span> |
| <span class="source-line-no">439</span><span id="line-439"> public long getTimestamp() {</span> |
| <span class="source-line-no">440</span><span id="line-440"> return ts;</span> |
| <span class="source-line-no">441</span><span id="line-441"> }</span> |
| <span class="source-line-no">442</span><span id="line-442"></span> |
| <span class="source-line-no">443</span><span id="line-443"> public void setTimestamp(long ts) {</span> |
| <span class="source-line-no">444</span><span id="line-444"> this.ts = ts;</span> |
| <span class="source-line-no">445</span><span id="line-445"> }</span> |
| <span class="source-line-no">446</span><span id="line-446"> }</span> |
| <span class="source-line-no">447</span><span id="line-447"></span> |
| <span class="source-line-no">448</span><span id="line-448"> public static class Loader extends Configured implements Tool {</span> |
| <span class="source-line-no">449</span><span id="line-449"></span> |
| <span class="source-line-no">450</span><span id="line-450"> private static final Logger LOG = LoggerFactory.getLogger(Loader.class);</span> |
| <span class="source-line-no">451</span><span id="line-451"> private static final String USAGE = "Loader <warInputDir | warFileList> <outputDir>";</span> |
| <span class="source-line-no">452</span><span id="line-452"></span> |
| <span class="source-line-no">453</span><span id="line-453"> void createSchema(final TableName tableName) throws IOException {</span> |
| <span class="source-line-no">454</span><span id="line-454"></span> |
| <span class="source-line-no">455</span><span id="line-455"> try (Connection conn = ConnectionFactory.createConnection(getConf());</span> |
| <span class="source-line-no">456</span><span id="line-456"> Admin admin = conn.getAdmin()) {</span> |
| <span class="source-line-no">457</span><span id="line-457"> if (!admin.tableExists(tableName)) {</span> |
| <span class="source-line-no">458</span><span id="line-458"></span> |
| <span class="source-line-no">459</span><span id="line-459"> ColumnFamilyDescriptorBuilder contentFamilyBuilder =</span> |
| <span class="source-line-no">460</span><span id="line-460"> ColumnFamilyDescriptorBuilder.newBuilder(CONTENT_FAMILY_NAME).setMaxVersions(1000)</span> |
| <span class="source-line-no">461</span><span id="line-461"> .setDataBlockEncoding(DataBlockEncoding.NONE).setBloomFilterType(BloomType.ROW);</span> |
| <span class="source-line-no">462</span><span id="line-462"></span> |
| <span class="source-line-no">463</span><span id="line-463"> ColumnFamilyDescriptorBuilder infoFamilyBuilder =</span> |
| <span class="source-line-no">464</span><span id="line-464"> ColumnFamilyDescriptorBuilder.newBuilder(INFO_FAMILY_NAME).setMaxVersions(1000)</span> |
| <span class="source-line-no">465</span><span id="line-465"> .setDataBlockEncoding(DataBlockEncoding.ROW_INDEX_V1)</span> |
| <span class="source-line-no">466</span><span id="line-466"> .setBloomFilterType(BloomType.ROWCOL).setBlocksize(8 * 1024);</span> |
| <span class="source-line-no">467</span><span id="line-467"></span> |
| <span class="source-line-no">468</span><span id="line-468"> ColumnFamilyDescriptorBuilder urlFamilyBuilder =</span> |
| <span class="source-line-no">469</span><span id="line-469"> ColumnFamilyDescriptorBuilder.newBuilder(URL_FAMILY_NAME).setMaxVersions(1000)</span> |
| <span class="source-line-no">470</span><span id="line-470"> .setDataBlockEncoding(DataBlockEncoding.ROW_INDEX_V1)</span> |
| <span class="source-line-no">471</span><span id="line-471"> .setBloomFilterType(BloomType.ROWCOL).setBlocksize(8 * 1024);</span> |
| <span class="source-line-no">472</span><span id="line-472"></span> |
| <span class="source-line-no">473</span><span id="line-473"> Set<ColumnFamilyDescriptor> families = new HashSet<>();</span> |
| <span class="source-line-no">474</span><span id="line-474"> families.add(contentFamilyBuilder.build());</span> |
| <span class="source-line-no">475</span><span id="line-475"> families.add(infoFamilyBuilder.build());</span> |
| <span class="source-line-no">476</span><span id="line-476"> families.add(urlFamilyBuilder.build());</span> |
| <span class="source-line-no">477</span><span id="line-477"></span> |
| <span class="source-line-no">478</span><span id="line-478"> TableDescriptor tableDescriptor =</span> |
| <span class="source-line-no">479</span><span id="line-479"> TableDescriptorBuilder.newBuilder(tableName).setColumnFamilies(families).build();</span> |
| <span class="source-line-no">480</span><span id="line-480"></span> |
| <span class="source-line-no">481</span><span id="line-481"> if (</span> |
| <span class="source-line-no">482</span><span id="line-482"> getConf().getBoolean(HBaseTestingUtil.PRESPLIT_TEST_TABLE_KEY,</span> |
| <span class="source-line-no">483</span><span id="line-483"> HBaseTestingUtil.PRESPLIT_TEST_TABLE)</span> |
| <span class="source-line-no">484</span><span id="line-484"> ) {</span> |
| <span class="source-line-no">485</span><span id="line-485"> int numberOfServers = admin.getRegionServers().size();</span> |
| <span class="source-line-no">486</span><span id="line-486"> if (numberOfServers == 0) {</span> |
| <span class="source-line-no">487</span><span id="line-487"> throw new IllegalStateException("No live regionservers");</span> |
| <span class="source-line-no">488</span><span id="line-488"> }</span> |
| <span class="source-line-no">489</span><span id="line-489"> int regionsPerServer = getConf().getInt(HBaseTestingUtil.REGIONS_PER_SERVER_KEY,</span> |
| <span class="source-line-no">490</span><span id="line-490"> HBaseTestingUtil.DEFAULT_REGIONS_PER_SERVER);</span> |
| <span class="source-line-no">491</span><span id="line-491"> int totalNumberOfRegions = numberOfServers * regionsPerServer;</span> |
| <span class="source-line-no">492</span><span id="line-492"> LOG.info("Creating test table: " + tableDescriptor);</span> |
| <span class="source-line-no">493</span><span id="line-493"> LOG.info("Number of live regionservers: " + numberOfServers + ", "</span> |
| <span class="source-line-no">494</span><span id="line-494"> + "pre-splitting table into " + totalNumberOfRegions + " regions "</span> |
| <span class="source-line-no">495</span><span id="line-495"> + "(default regions per server: " + regionsPerServer + ")");</span> |
| <span class="source-line-no">496</span><span id="line-496"> byte[][] splits = new RegionSplitter.UniformSplit().split(totalNumberOfRegions);</span> |
| <span class="source-line-no">497</span><span id="line-497"> admin.createTable(tableDescriptor, splits);</span> |
| <span class="source-line-no">498</span><span id="line-498"> } else {</span> |
| <span class="source-line-no">499</span><span id="line-499"> LOG.info("Creating test table: " + tableDescriptor);</span> |
| <span class="source-line-no">500</span><span id="line-500"> admin.createTable(tableDescriptor);</span> |
| <span class="source-line-no">501</span><span id="line-501"> }</span> |
| <span class="source-line-no">502</span><span id="line-502"> }</span> |
| <span class="source-line-no">503</span><span id="line-503"> } catch (MasterNotRunningException e) {</span> |
| <span class="source-line-no">504</span><span id="line-504"> LOG.error("Master not running", e);</span> |
| <span class="source-line-no">505</span><span id="line-505"> throw new IOException(e);</span> |
| <span class="source-line-no">506</span><span id="line-506"> }</span> |
| <span class="source-line-no">507</span><span id="line-507"> }</span> |
| <span class="source-line-no">508</span><span id="line-508"></span> |
| <span class="source-line-no">509</span><span id="line-509"> int run(final Path warcFileInput, final Path outputDir)</span> |
| <span class="source-line-no">510</span><span id="line-510"> throws IOException, ClassNotFoundException, InterruptedException {</span> |
| <span class="source-line-no">511</span><span id="line-511"></span> |
| <span class="source-line-no">512</span><span id="line-512"> createSchema(getTablename(getConf()));</span> |
| <span class="source-line-no">513</span><span id="line-513"></span> |
| <span class="source-line-no">514</span><span id="line-514"> final Job job = Job.getInstance(getConf());</span> |
| <span class="source-line-no">515</span><span id="line-515"> job.setJobName(Loader.class.getName());</span> |
| <span class="source-line-no">516</span><span id="line-516"> job.setNumReduceTasks(0);</span> |
| <span class="source-line-no">517</span><span id="line-517"> job.setJarByClass(getClass());</span> |
| <span class="source-line-no">518</span><span id="line-518"> job.setMapperClass(LoaderMapper.class);</span> |
| <span class="source-line-no">519</span><span id="line-519"> job.setInputFormatClass(WARCInputFormat.class);</span> |
| <span class="source-line-no">520</span><span id="line-520"> final FileSystem fs = FileSystem.get(warcFileInput.toUri(), getConf());</span> |
| <span class="source-line-no">521</span><span id="line-521"> if (fs.getFileStatus(warcFileInput).isDirectory()) {</span> |
| <span class="source-line-no">522</span><span id="line-522"> LOG.info("Using directory as WARC input path: " + warcFileInput);</span> |
| <span class="source-line-no">523</span><span id="line-523"> FileInputFormat.setInputPaths(job, warcFileInput);</span> |
| <span class="source-line-no">524</span><span id="line-524"> } else if (warcFileInput.toUri().getScheme().equals("file")) {</span> |
| <span class="source-line-no">525</span><span id="line-525"> LOG.info("Getting WARC input paths from file: " + warcFileInput);</span> |
| <span class="source-line-no">526</span><span id="line-526"> final List<Path> paths = new ArrayList<Path>();</span> |
| <span class="source-line-no">527</span><span id="line-527"> try (FSDataInputStream is = fs.open(warcFileInput)) {</span> |
| <span class="source-line-no">528</span><span id="line-528"> InputStreamReader reader;</span> |
| <span class="source-line-no">529</span><span id="line-529"> if (warcFileInput.getName().toLowerCase().endsWith(".gz")) {</span> |
| <span class="source-line-no">530</span><span id="line-530"> reader = new InputStreamReader(new GZIPInputStream(is), StandardCharsets.UTF_8);</span> |
| <span class="source-line-no">531</span><span id="line-531"> } else {</span> |
| <span class="source-line-no">532</span><span id="line-532"> reader = new InputStreamReader(is, StandardCharsets.UTF_8);</span> |
| <span class="source-line-no">533</span><span id="line-533"> }</span> |
| <span class="source-line-no">534</span><span id="line-534"> try (BufferedReader br = new BufferedReader(reader)) {</span> |
| <span class="source-line-no">535</span><span id="line-535"> String line;</span> |
| <span class="source-line-no">536</span><span id="line-536"> while ((line = br.readLine()) != null) {</span> |
| <span class="source-line-no">537</span><span id="line-537"> paths.add(new Path(line));</span> |
| <span class="source-line-no">538</span><span id="line-538"> }</span> |
| <span class="source-line-no">539</span><span id="line-539"> }</span> |
| <span class="source-line-no">540</span><span id="line-540"> }</span> |
| <span class="source-line-no">541</span><span id="line-541"> LOG.info("Read " + paths.size() + " WARC input paths from " + warcFileInput);</span> |
| <span class="source-line-no">542</span><span id="line-542"> FileInputFormat.setInputPaths(job, paths.toArray(new Path[paths.size()]));</span> |
| <span class="source-line-no">543</span><span id="line-543"> } else {</span> |
| <span class="source-line-no">544</span><span id="line-544"> FileInputFormat.setInputPaths(job, warcFileInput);</span> |
| <span class="source-line-no">545</span><span id="line-545"> }</span> |
| <span class="source-line-no">546</span><span id="line-546"> job.setOutputFormatClass(SequenceFileOutputFormat.class);</span> |
| <span class="source-line-no">547</span><span id="line-547"> SequenceFileOutputFormat.setOutputPath(job, outputDir);</span> |
| <span class="source-line-no">548</span><span id="line-548"> SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);</span> |
| <span class="source-line-no">549</span><span id="line-549"> job.setOutputKeyClass(HBaseKeyWritable.class);</span> |
| <span class="source-line-no">550</span><span id="line-550"> job.setOutputValueClass(BytesWritable.class);</span> |
| <span class="source-line-no">551</span><span id="line-551"> TableMapReduceUtil.addDependencyJars(job);</span> |
| <span class="source-line-no">552</span><span id="line-552"> // Increase max attempts because S3 might throttle aggressively and ultimately fail a task</span> |
| <span class="source-line-no">553</span><span id="line-553"> job.getConfiguration().setInt("mapred.map.max.attempts", 100);</span> |
| <span class="source-line-no">554</span><span id="line-554"> job.getConfiguration().setInt("mapreduce.map.maxattempts", 100);</span> |
| <span class="source-line-no">555</span><span id="line-555"></span> |
| <span class="source-line-no">556</span><span id="line-556"> boolean success = job.waitForCompletion(true);</span> |
| <span class="source-line-no">557</span><span id="line-557"> if (!success) {</span> |
| <span class="source-line-no">558</span><span id="line-558"> LOG.error("Failure during job " + job.getJobID());</span> |
| <span class="source-line-no">559</span><span id="line-559"> }</span> |
| <span class="source-line-no">560</span><span id="line-560"></span> |
| <span class="source-line-no">561</span><span id="line-561"> final Counters counters = job.getCounters();</span> |
| <span class="source-line-no">562</span><span id="line-562"> for (Counts c : Counts.values()) {</span> |
| <span class="source-line-no">563</span><span id="line-563"> long value = counters.findCounter(c).getValue();</span> |
| <span class="source-line-no">564</span><span id="line-564"> if (value != 0) {</span> |
| <span class="source-line-no">565</span><span id="line-565"> LOG.info(c + ": " + value);</span> |
| <span class="source-line-no">566</span><span id="line-566"> }</span> |
| <span class="source-line-no">567</span><span id="line-567"> }</span> |
| <span class="source-line-no">568</span><span id="line-568"></span> |
| <span class="source-line-no">569</span><span id="line-569"> return success ? 0 : 1;</span> |
| <span class="source-line-no">570</span><span id="line-570"> }</span> |
| <span class="source-line-no">571</span><span id="line-571"></span> |
| <span class="source-line-no">572</span><span id="line-572"> @Override</span> |
| <span class="source-line-no">573</span><span id="line-573"> public int run(String[] args) throws Exception {</span> |
| <span class="source-line-no">574</span><span id="line-574"> if (args.length < 2) {</span> |
| <span class="source-line-no">575</span><span id="line-575"> System.err.println(USAGE);</span> |
| <span class="source-line-no">576</span><span id="line-576"> return 1;</span> |
| <span class="source-line-no">577</span><span id="line-577"> }</span> |
| <span class="source-line-no">578</span><span id="line-578"> try {</span> |
| <span class="source-line-no">579</span><span id="line-579"> Path warcFileInput = new Path(args[0]);</span> |
| <span class="source-line-no">580</span><span id="line-580"> Path outputDir = new Path(args[1]);</span> |
| <span class="source-line-no">581</span><span id="line-581"> return run(warcFileInput, outputDir);</span> |
| <span class="source-line-no">582</span><span id="line-582"> } catch (NumberFormatException e) {</span> |
| <span class="source-line-no">583</span><span id="line-583"> System.err.println("Parsing loader arguments failed: " + e.getMessage());</span> |
| <span class="source-line-no">584</span><span id="line-584"> System.err.println(USAGE);</span> |
| <span class="source-line-no">585</span><span id="line-585"> return 1;</span> |
| <span class="source-line-no">586</span><span id="line-586"> }</span> |
| <span class="source-line-no">587</span><span id="line-587"> }</span> |
| <span class="source-line-no">588</span><span id="line-588"></span> |
| <span class="source-line-no">589</span><span id="line-589"> public static void main(String[] args) throws Exception {</span> |
| <span class="source-line-no">590</span><span id="line-590"> System.exit(ToolRunner.run(HBaseConfiguration.create(), new Loader(), args));</span> |
| <span class="source-line-no">591</span><span id="line-591"> }</span> |
| <span class="source-line-no">592</span><span id="line-592"></span> |
| <span class="source-line-no">593</span><span id="line-593"> public static class LoaderMapper</span> |
| <span class="source-line-no">594</span><span id="line-594"> extends Mapper<LongWritable, WARCWritable, HBaseKeyWritable, BytesWritable> {</span> |
| <span class="source-line-no">595</span><span id="line-595"></span> |
| <span class="source-line-no">596</span><span id="line-596"> protected AsyncConnection conn;</span> |
| <span class="source-line-no">597</span><span id="line-597"> protected AsyncTable<ScanResultConsumer> table;</span> |
| <span class="source-line-no">598</span><span id="line-598"> protected ExecutorService executor;</span> |
| <span class="source-line-no">599</span><span id="line-599"> protected AtomicLong inflight = new AtomicLong();</span> |
| <span class="source-line-no">600</span><span id="line-600"> protected boolean doIncrements;</span> |
| <span class="source-line-no">601</span><span id="line-601"></span> |
| <span class="source-line-no">602</span><span id="line-602"> @Override</span> |
| <span class="source-line-no">603</span><span id="line-603"> protected void setup(final Context context) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">604</span><span id="line-604"> executor = Executors.newWorkStealingPool(Runtime.getRuntime().availableProcessors());</span> |
| <span class="source-line-no">605</span><span id="line-605"> Configuration conf = context.getConfiguration();</span> |
| <span class="source-line-no">606</span><span id="line-606"> doIncrements = conf.getBoolean(INCREMENTS_NAME_KEY, DEFAULT_INCREMENTS);</span> |
| <span class="source-line-no">607</span><span id="line-607"> try {</span> |
| <span class="source-line-no">608</span><span id="line-608"> conn = ConnectionFactory.createAsyncConnection(conf).get();</span> |
| <span class="source-line-no">609</span><span id="line-609"> table = conn.getTable(getTablename(conf), executor);</span> |
| <span class="source-line-no">610</span><span id="line-610"> } catch (ExecutionException e) {</span> |
| <span class="source-line-no">611</span><span id="line-611"> throw new IOException(e);</span> |
| <span class="source-line-no">612</span><span id="line-612"> }</span> |
| <span class="source-line-no">613</span><span id="line-613"> }</span> |
| <span class="source-line-no">614</span><span id="line-614"></span> |
| <span class="source-line-no">615</span><span id="line-615"> @Override</span> |
| <span class="source-line-no">616</span><span id="line-616"> protected void cleanup(final Context context) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">617</span><span id="line-617"></span> |
| <span class="source-line-no">618</span><span id="line-618"> while (inflight.get() != 0) {</span> |
| <span class="source-line-no">619</span><span id="line-619"> LOG.info("Operations in flight, waiting");</span> |
| <span class="source-line-no">620</span><span id="line-620"> Thread.sleep(INFLIGHT_PAUSE_MS);</span> |
| <span class="source-line-no">621</span><span id="line-621"> }</span> |
| <span class="source-line-no">622</span><span id="line-622"></span> |
| <span class="source-line-no">623</span><span id="line-623"> // Shut down the executor</span> |
| <span class="source-line-no">624</span><span id="line-624"> executor.shutdown();</span> |
| <span class="source-line-no">625</span><span id="line-625"> if (!executor.awaitTermination(1, TimeUnit.MINUTES)) {</span> |
| <span class="source-line-no">626</span><span id="line-626"> LOG.warn("Pool did not shut down cleanly");</span> |
| <span class="source-line-no">627</span><span id="line-627"> }</span> |
| <span class="source-line-no">628</span><span id="line-628"> // Close the connection</span> |
| <span class="source-line-no">629</span><span id="line-629"> try {</span> |
| <span class="source-line-no">630</span><span id="line-630"> conn.close();</span> |
| <span class="source-line-no">631</span><span id="line-631"> } catch (Exception e) {</span> |
| <span class="source-line-no">632</span><span id="line-632"> LOG.warn("Exception closing Connection", e);</span> |
| <span class="source-line-no">633</span><span id="line-633"> }</span> |
| <span class="source-line-no">634</span><span id="line-634"> }</span> |
| <span class="source-line-no">635</span><span id="line-635"></span> |
| <span class="source-line-no">636</span><span id="line-636"> @Override</span> |
| <span class="source-line-no">637</span><span id="line-637"> protected void map(final LongWritable key, final WARCWritable value, final Context output)</span> |
| <span class="source-line-no">638</span><span id="line-638"> throws IOException, InterruptedException {</span> |
| <span class="source-line-no">639</span><span id="line-639"> final WARCRecord.Header warcHeader = value.getRecord().getHeader();</span> |
| <span class="source-line-no">640</span><span id="line-640"> final String recordID = warcHeader.getRecordID();</span> |
| <span class="source-line-no">641</span><span id="line-641"> final String targetURI = warcHeader.getTargetURI();</span> |
| <span class="source-line-no">642</span><span id="line-642"> if (warcHeader.getRecordType().equals("response") && targetURI != null) {</span> |
| <span class="source-line-no">643</span><span id="line-643"> final String contentType = warcHeader.getField("WARC-Identified-Payload-Type");</span> |
| <span class="source-line-no">644</span><span id="line-644"> if (contentType != null) {</span> |
| <span class="source-line-no">645</span><span id="line-645"> // Make row key</span> |
| <span class="source-line-no">646</span><span id="line-646"> byte[] rowKey;</span> |
| <span class="source-line-no">647</span><span id="line-647"> try {</span> |
| <span class="source-line-no">648</span><span id="line-648"> rowKey = rowKeyFromTargetURI(targetURI);</span> |
| <span class="source-line-no">649</span><span id="line-649"> } catch (IllegalArgumentException e) {</span> |
| <span class="source-line-no">650</span><span id="line-650"> LOG.debug("Could not make a row key for record " + recordID + ", ignoring", e);</span> |
| <span class="source-line-no">651</span><span id="line-651"> return;</span> |
| <span class="source-line-no">652</span><span id="line-652"> } catch (URISyntaxException e) {</span> |
| <span class="source-line-no">653</span><span id="line-653"> LOG.warn(</span> |
| <span class="source-line-no">654</span><span id="line-654"> "Could not parse URI \"" + targetURI + "\" for record " + recordID + ", ignoring");</span> |
| <span class="source-line-no">655</span><span id="line-655"> return;</span> |
| <span class="source-line-no">656</span><span id="line-656"> }</span> |
| <span class="source-line-no">657</span><span id="line-657"></span> |
| <span class="source-line-no">658</span><span id="line-658"> // Get the content and calculate the CRC64</span> |
| <span class="source-line-no">659</span><span id="line-659"> final byte[] content = value.getRecord().getContent();</span> |
| <span class="source-line-no">660</span><span id="line-660"> final CRC64 crc = new CRC64();</span> |
| <span class="source-line-no">661</span><span id="line-661"> crc.update(content);</span> |
| <span class="source-line-no">662</span><span id="line-662"> final long crc64 = crc.getValue();</span> |
| <span class="source-line-no">663</span><span id="line-663"> LOG.info("{}: content {} bytes, crc64={}", targetURI, content.length,</span> |
| <span class="source-line-no">664</span><span id="line-664"> Bytes.toHex(Bytes.toBytes(crc64)));</span> |
| <span class="source-line-no">665</span><span id="line-665"></span> |
| <span class="source-line-no">666</span><span id="line-666"> // Store to HBase</span> |
| <span class="source-line-no">667</span><span id="line-667"> final long ts = getSequence();</span> |
| <span class="source-line-no">668</span><span id="line-668"> final Put put = new Put(rowKey);</span> |
| <span class="source-line-no">669</span><span id="line-669"> put.addColumn(CONTENT_FAMILY_NAME, CONTENT_QUALIFIER, ts, content);</span> |
| <span class="source-line-no">670</span><span id="line-670"> put.addColumn(INFO_FAMILY_NAME, CONTENT_LENGTH_QUALIFIER, ts,</span> |
| <span class="source-line-no">671</span><span id="line-671"> Bytes.toBytes(content.length));</span> |
| <span class="source-line-no">672</span><span id="line-672"> put.addColumn(INFO_FAMILY_NAME, CONTENT_TYPE_QUALIFIER, ts, Bytes.toBytes(contentType));</span> |
| <span class="source-line-no">673</span><span id="line-673"> put.addColumn(INFO_FAMILY_NAME, CRC_QUALIFIER, ts, Bytes.toBytes(crc64));</span> |
| <span class="source-line-no">674</span><span id="line-674"> put.addColumn(INFO_FAMILY_NAME, TARGET_URI_QUALIFIER, ts, Bytes.toBytes(targetURI));</span> |
| <span class="source-line-no">675</span><span id="line-675"> put.addColumn(INFO_FAMILY_NAME, DATE_QUALIFIER, ts,</span> |
| <span class="source-line-no">676</span><span id="line-676"> Bytes.toBytes(warcHeader.getDateString()));</span> |
| <span class="source-line-no">677</span><span id="line-677"> final String ipAddr = warcHeader.getField("WARC-IP-Address");</span> |
| <span class="source-line-no">678</span><span id="line-678"> if (ipAddr != null) {</span> |
| <span class="source-line-no">679</span><span id="line-679"> put.addColumn(INFO_FAMILY_NAME, IP_ADDRESS_QUALIFIER, ts, Bytes.toBytes(ipAddr));</span> |
| <span class="source-line-no">680</span><span id="line-680"> }</span> |
| <span class="source-line-no">681</span><span id="line-681"> long pending = inflight.incrementAndGet();</span> |
| <span class="source-line-no">682</span><span id="line-682"> while (pending > MAX_INFLIGHT) {</span> |
| <span class="source-line-no">683</span><span id="line-683"> LOG.info("Too many operations in flight, waiting");</span> |
| <span class="source-line-no">684</span><span id="line-684"> Thread.sleep(INFLIGHT_PAUSE_MS);</span> |
| <span class="source-line-no">685</span><span id="line-685"> pending = inflight.get();</span> |
| <span class="source-line-no">686</span><span id="line-686"> }</span> |
| <span class="source-line-no">687</span><span id="line-687"> final long putStartTime = System.currentTimeMillis();</span> |
| <span class="source-line-no">688</span><span id="line-688"> final CompletableFuture<Void> putFuture = table.put(put);</span> |
| <span class="source-line-no">689</span><span id="line-689"> addListener(putFuture, (r, e) -> {</span> |
| <span class="source-line-no">690</span><span id="line-690"> inflight.decrementAndGet();</span> |
| <span class="source-line-no">691</span><span id="line-691"> if (e == null) {</span> |
| <span class="source-line-no">692</span><span id="line-692"> output.getCounter(Counts.RPC_TIME_MS)</span> |
| <span class="source-line-no">693</span><span id="line-693"> .increment(System.currentTimeMillis() - putStartTime);</span> |
| <span class="source-line-no">694</span><span id="line-694"> output.getCounter(Counts.RPC_BYTES_WRITTEN).increment(put.heapSize());</span> |
| <span class="source-line-no">695</span><span id="line-695"> }</span> |
| <span class="source-line-no">696</span><span id="line-696"> });</span> |
| <span class="source-line-no">697</span><span id="line-697"></span> |
| <span class="source-line-no">698</span><span id="line-698"> // Write records out for later verification, one per HBase field except for the</span> |
| <span class="source-line-no">699</span><span id="line-699"> // content record, which will be verified by CRC64.</span> |
| <span class="source-line-no">700</span><span id="line-700"> output.write(</span> |
| <span class="source-line-no">701</span><span id="line-701"> new HBaseKeyWritable(rowKey, INFO_FAMILY_NAME, CONTENT_LENGTH_QUALIFIER, ts),</span> |
| <span class="source-line-no">702</span><span id="line-702"> new BytesWritable(Bytes.toBytes(content.length)));</span> |
| <span class="source-line-no">703</span><span id="line-703"> output.write(new HBaseKeyWritable(rowKey, INFO_FAMILY_NAME, CONTENT_TYPE_QUALIFIER, ts),</span> |
| <span class="source-line-no">704</span><span id="line-704"> new BytesWritable(Bytes.toBytes(contentType)));</span> |
| <span class="source-line-no">705</span><span id="line-705"> output.write(new HBaseKeyWritable(rowKey, INFO_FAMILY_NAME, CRC_QUALIFIER, ts),</span> |
| <span class="source-line-no">706</span><span id="line-706"> new BytesWritable(Bytes.toBytes(crc64)));</span> |
| <span class="source-line-no">707</span><span id="line-707"> output.write(new HBaseKeyWritable(rowKey, INFO_FAMILY_NAME, TARGET_URI_QUALIFIER, ts),</span> |
| <span class="source-line-no">708</span><span id="line-708"> new BytesWritable(Bytes.toBytes(targetURI)));</span> |
| <span class="source-line-no">709</span><span id="line-709"> output.write(new HBaseKeyWritable(rowKey, INFO_FAMILY_NAME, DATE_QUALIFIER, ts),</span> |
| <span class="source-line-no">710</span><span id="line-710"> new BytesWritable(Bytes.toBytes(warcHeader.getDateString())));</span> |
| <span class="source-line-no">711</span><span id="line-711"> if (ipAddr != null) {</span> |
| <span class="source-line-no">712</span><span id="line-712"> output.write(new HBaseKeyWritable(rowKey, INFO_FAMILY_NAME, IP_ADDRESS_QUALIFIER, ts),</span> |
| <span class="source-line-no">713</span><span id="line-713"> new BytesWritable(Bytes.toBytes(ipAddr)));</span> |
| <span class="source-line-no">714</span><span id="line-714"> }</span> |
| <span class="source-line-no">715</span><span id="line-715"></span> |
| <span class="source-line-no">716</span><span id="line-716"> if (doIncrements) {</span> |
| <span class="source-line-no">717</span><span id="line-717"> // The URLs cf is not tracked for correctness. For now it is used only to exercise</span> |
| <span class="source-line-no">718</span><span id="line-718"> // Increments, to drive some read load during ingest. They can be verified with a</span> |
| <span class="source-line-no">719</span><span id="line-719"> // reducer to sum increments per row and then compare the final count to the table</span> |
| <span class="source-line-no">720</span><span id="line-720"> // data. This is left as a future exercise.</span> |
| <span class="source-line-no">721</span><span id="line-721"> final byte[] refQual = Bytes.add(REF_QUALIFIER, SEP, rowKey);</span> |
| <span class="source-line-no">722</span><span id="line-722"> for (String refUri : extractUrls(content)) {</span> |
| <span class="source-line-no">723</span><span id="line-723"> try {</span> |
| <span class="source-line-no">724</span><span id="line-724"> byte[] urlRowKey = rowKeyFromTargetURI(refUri);</span> |
| <span class="source-line-no">725</span><span id="line-725"> LOG.debug(" -> {}", refUri);</span> |
| <span class="source-line-no">726</span><span id="line-726"> final Increment increment = new Increment(urlRowKey);</span> |
| <span class="source-line-no">727</span><span id="line-727"> increment.setTimestamp(ts);</span> |
| <span class="source-line-no">728</span><span id="line-728"> increment.addColumn(URL_FAMILY_NAME, refQual, 1);</span> |
| <span class="source-line-no">729</span><span id="line-729"> pending = inflight.incrementAndGet();</span> |
| <span class="source-line-no">730</span><span id="line-730"> while (pending > MAX_INFLIGHT) {</span> |
| <span class="source-line-no">731</span><span id="line-731"> LOG.info("Too many operations in flight, waiting");</span> |
| <span class="source-line-no">732</span><span id="line-732"> Thread.sleep(INFLIGHT_PAUSE_MS);</span> |
| <span class="source-line-no">733</span><span id="line-733"> pending = inflight.get();</span> |
| <span class="source-line-no">734</span><span id="line-734"> }</span> |
| <span class="source-line-no">735</span><span id="line-735"> final long incrStartTime = System.currentTimeMillis();</span> |
| <span class="source-line-no">736</span><span id="line-736"> final CompletableFuture<Result> incrFuture = table.increment(increment);</span> |
| <span class="source-line-no">737</span><span id="line-737"> addListener(incrFuture, (r, e) -> {</span> |
| <span class="source-line-no">738</span><span id="line-738"> inflight.decrementAndGet();</span> |
| <span class="source-line-no">739</span><span id="line-739"> if (e == null) {</span> |
| <span class="source-line-no">740</span><span id="line-740"> output.getCounter(Counts.RPC_TIME_MS)</span> |
| <span class="source-line-no">741</span><span id="line-741"> .increment(System.currentTimeMillis() - incrStartTime);</span> |
| <span class="source-line-no">742</span><span id="line-742"> output.getCounter(Counts.RPC_BYTES_WRITTEN).increment(increment.heapSize());</span> |
| <span class="source-line-no">743</span><span id="line-743"> }</span> |
| <span class="source-line-no">744</span><span id="line-744"> });</span> |
| <span class="source-line-no">745</span><span id="line-745"> } catch (IllegalArgumentException | URISyntaxException e) {</span> |
| <span class="source-line-no">746</span><span id="line-746"> LOG.debug("Could not make a row key for URI " + refUri + ", ignoring", e);</span> |
| <span class="source-line-no">747</span><span id="line-747"> }</span> |
| <span class="source-line-no">748</span><span id="line-748"> }</span> |
| <span class="source-line-no">749</span><span id="line-749"> }</span> |
| <span class="source-line-no">750</span><span id="line-750"> }</span> |
| <span class="source-line-no">751</span><span id="line-751"> }</span> |
| <span class="source-line-no">752</span><span id="line-752"> }</span> |
| <span class="source-line-no">753</span><span id="line-753"> }</span> |
| <span class="source-line-no">754</span><span id="line-754"> }</span> |
| <span class="source-line-no">755</span><span id="line-755"></span> |
| <span class="source-line-no">756</span><span id="line-756"> public static class OneFilePerMapperSFIF<K, V> extends SequenceFileInputFormat<K, V> {</span> |
| <span class="source-line-no">757</span><span id="line-757"> @Override</span> |
| <span class="source-line-no">758</span><span id="line-758"> protected boolean isSplitable(final JobContext context, final Path filename) {</span> |
| <span class="source-line-no">759</span><span id="line-759"> return false;</span> |
| <span class="source-line-no">760</span><span id="line-760"> }</span> |
| <span class="source-line-no">761</span><span id="line-761"> }</span> |
| <span class="source-line-no">762</span><span id="line-762"></span> |
| <span class="source-line-no">763</span><span id="line-763"> public static class Verify extends Configured implements Tool {</span> |
| <span class="source-line-no">764</span><span id="line-764"></span> |
| <span class="source-line-no">765</span><span id="line-765"> public static final Logger LOG = LoggerFactory.getLogger(Verify.class);</span> |
| <span class="source-line-no">766</span><span id="line-766"> public static final String USAGE = "Verify <inputDir>";</span> |
| <span class="source-line-no">767</span><span id="line-767"></span> |
| <span class="source-line-no">768</span><span id="line-768"> int run(final Path inputDir) throws IOException, ClassNotFoundException, InterruptedException {</span> |
| <span class="source-line-no">769</span><span id="line-769"> Job job = Job.getInstance(getConf());</span> |
| <span class="source-line-no">770</span><span id="line-770"> job.setJobName(Verify.class.getName());</span> |
| <span class="source-line-no">771</span><span id="line-771"> job.setJarByClass(getClass());</span> |
| <span class="source-line-no">772</span><span id="line-772"> job.setMapperClass(VerifyMapper.class);</span> |
| <span class="source-line-no">773</span><span id="line-773"> job.setInputFormatClass(OneFilePerMapperSFIF.class);</span> |
| <span class="source-line-no">774</span><span id="line-774"> FileInputFormat.setInputPaths(job, inputDir);</span> |
| <span class="source-line-no">775</span><span id="line-775"> job.setOutputFormatClass(NullOutputFormat.class);</span> |
| <span class="source-line-no">776</span><span id="line-776"> job.setOutputKeyClass(NullWritable.class);</span> |
| <span class="source-line-no">777</span><span id="line-777"> job.setOutputValueClass(NullWritable.class);</span> |
| <span class="source-line-no">778</span><span id="line-778"> TableMapReduceUtil.addDependencyJars(job);</span> |
| <span class="source-line-no">779</span><span id="line-779"></span> |
| <span class="source-line-no">780</span><span id="line-780"> boolean success = job.waitForCompletion(true);</span> |
| <span class="source-line-no">781</span><span id="line-781"> if (!success) {</span> |
| <span class="source-line-no">782</span><span id="line-782"> LOG.error("Failure during job " + job.getJobID());</span> |
| <span class="source-line-no">783</span><span id="line-783"> }</span> |
| <span class="source-line-no">784</span><span id="line-784"></span> |
| <span class="source-line-no">785</span><span id="line-785"> final Counters counters = job.getCounters();</span> |
| <span class="source-line-no">786</span><span id="line-786"> for (Counts c : Counts.values()) {</span> |
| <span class="source-line-no">787</span><span id="line-787"> long value = counters.findCounter(c).getValue();</span> |
| <span class="source-line-no">788</span><span id="line-788"> if (value != 0) {</span> |
| <span class="source-line-no">789</span><span id="line-789"> LOG.info(c + ": " + value);</span> |
| <span class="source-line-no">790</span><span id="line-790"> }</span> |
| <span class="source-line-no">791</span><span id="line-791"> }</span> |
| <span class="source-line-no">792</span><span id="line-792"> if (counters.findCounter(Counts.UNREFERENCED).getValue() > 0) {</span> |
| <span class="source-line-no">793</span><span id="line-793"> LOG.error("Nonzero UNREFERENCED count from job " + job.getJobID());</span> |
| <span class="source-line-no">794</span><span id="line-794"> success = false;</span> |
| <span class="source-line-no">795</span><span id="line-795"> }</span> |
| <span class="source-line-no">796</span><span id="line-796"> if (counters.findCounter(Counts.CORRUPT).getValue() > 0) {</span> |
| <span class="source-line-no">797</span><span id="line-797"> LOG.error("Nonzero CORRUPT count from job " + job.getJobID());</span> |
| <span class="source-line-no">798</span><span id="line-798"> success = false;</span> |
| <span class="source-line-no">799</span><span id="line-799"> }</span> |
| <span class="source-line-no">800</span><span id="line-800"></span> |
| <span class="source-line-no">801</span><span id="line-801"> return success ? 0 : 1;</span> |
| <span class="source-line-no">802</span><span id="line-802"> }</span> |
| <span class="source-line-no">803</span><span id="line-803"></span> |
| <span class="source-line-no">804</span><span id="line-804"> @Override</span> |
| <span class="source-line-no">805</span><span id="line-805"> public int run(String[] args) throws Exception {</span> |
| <span class="source-line-no">806</span><span id="line-806"> if (args.length < 2) {</span> |
| <span class="source-line-no">807</span><span id="line-807"> System.err.println(USAGE);</span> |
| <span class="source-line-no">808</span><span id="line-808"> return 1;</span> |
| <span class="source-line-no">809</span><span id="line-809"> }</span> |
| <span class="source-line-no">810</span><span id="line-810"> Path loaderOutput = new Path(args[0]);</span> |
| <span class="source-line-no">811</span><span id="line-811"> return run(loaderOutput);</span> |
| <span class="source-line-no">812</span><span id="line-812"> }</span> |
| <span class="source-line-no">813</span><span id="line-813"></span> |
| <span class="source-line-no">814</span><span id="line-814"> public static void main(String[] args) throws Exception {</span> |
| <span class="source-line-no">815</span><span id="line-815"> System.exit(ToolRunner.run(HBaseConfiguration.create(), new Verify(), args));</span> |
| <span class="source-line-no">816</span><span id="line-816"> }</span> |
| <span class="source-line-no">817</span><span id="line-817"></span> |
| <span class="source-line-no">818</span><span id="line-818"> public static class VerifyMapper</span> |
| <span class="source-line-no">819</span><span id="line-819"> extends Mapper<HBaseKeyWritable, BytesWritable, NullWritable, NullWritable> {</span> |
| <span class="source-line-no">820</span><span id="line-820"></span> |
| <span class="source-line-no">821</span><span id="line-821"> protected Connection conn;</span> |
| <span class="source-line-no">822</span><span id="line-822"> protected Table table;</span> |
| <span class="source-line-no">823</span><span id="line-823"></span> |
| <span class="source-line-no">824</span><span id="line-824"> @Override</span> |
| <span class="source-line-no">825</span><span id="line-825"> protected void setup(final Context context) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">826</span><span id="line-826"> Configuration conf = context.getConfiguration();</span> |
| <span class="source-line-no">827</span><span id="line-827"> conn = ConnectionFactory.createConnection(conf);</span> |
| <span class="source-line-no">828</span><span id="line-828"> table = conn.getTable(getTablename(conf));</span> |
| <span class="source-line-no">829</span><span id="line-829"> }</span> |
| <span class="source-line-no">830</span><span id="line-830"></span> |
| <span class="source-line-no">831</span><span id="line-831"> @Override</span> |
| <span class="source-line-no">832</span><span id="line-832"> protected void cleanup(final Context context) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">833</span><span id="line-833"> // Close the table</span> |
| <span class="source-line-no">834</span><span id="line-834"> try {</span> |
| <span class="source-line-no">835</span><span id="line-835"> table.close();</span> |
| <span class="source-line-no">836</span><span id="line-836"> } catch (Exception e) {</span> |
| <span class="source-line-no">837</span><span id="line-837"> LOG.warn("Exception closing table", e);</span> |
| <span class="source-line-no">838</span><span id="line-838"> }</span> |
| <span class="source-line-no">839</span><span id="line-839"> // Close the connection</span> |
| <span class="source-line-no">840</span><span id="line-840"> try {</span> |
| <span class="source-line-no">841</span><span id="line-841"> conn.close();</span> |
| <span class="source-line-no">842</span><span id="line-842"> } catch (Exception e) {</span> |
| <span class="source-line-no">843</span><span id="line-843"> LOG.warn("Exception closing Connection", e);</span> |
| <span class="source-line-no">844</span><span id="line-844"> }</span> |
| <span class="source-line-no">845</span><span id="line-845"> }</span> |
| <span class="source-line-no">846</span><span id="line-846"></span> |
| <span class="source-line-no">847</span><span id="line-847"> @Override</span> |
| <span class="source-line-no">848</span><span id="line-848"> protected void map(final HBaseKeyWritable key, final BytesWritable value,</span> |
| <span class="source-line-no">849</span><span id="line-849"> final Context output) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">850</span><span id="line-850"> final byte[] row = Bytes.copy(key.getRowArray(), key.getRowOffset(), key.getRowLength());</span> |
| <span class="source-line-no">851</span><span id="line-851"> final byte[] family =</span> |
| <span class="source-line-no">852</span><span id="line-852"> Bytes.copy(key.getFamilyArray(), key.getFamilyOffset(), key.getFamilyLength());</span> |
| <span class="source-line-no">853</span><span id="line-853"> final byte[] qualifier =</span> |
| <span class="source-line-no">854</span><span id="line-854"> Bytes.copy(key.getQualifierArray(), key.getQualifierOffset(), key.getQualifierLength());</span> |
| <span class="source-line-no">855</span><span id="line-855"> final long ts = key.getTimestamp();</span> |
| <span class="source-line-no">856</span><span id="line-856"></span> |
| <span class="source-line-no">857</span><span id="line-857"> if (Bytes.equals(INFO_FAMILY_NAME, family) && Bytes.equals(CRC_QUALIFIER, qualifier)) {</span> |
| <span class="source-line-no">858</span><span id="line-858"> final long expectedCRC64 = Bytes.toLong(value.getBytes(), 0, value.getLength());</span> |
| <span class="source-line-no">859</span><span id="line-859"> final Get get = new Get(row).setTimestamp(ts).addFamily(CONTENT_FAMILY_NAME)</span> |
| <span class="source-line-no">860</span><span id="line-860"> .addFamily(INFO_FAMILY_NAME);</span> |
| <span class="source-line-no">861</span><span id="line-861"> final long startTime = System.currentTimeMillis();</span> |
| <span class="source-line-no">862</span><span id="line-862"> Result r;</span> |
| <span class="source-line-no">863</span><span id="line-863"> try {</span> |
| <span class="source-line-no">864</span><span id="line-864"> r = table.get(get);</span> |
| <span class="source-line-no">865</span><span id="line-865"> output.getCounter(Counts.RPC_TIME_MS).increment(System.currentTimeMillis() - startTime);</span> |
| <span class="source-line-no">866</span><span id="line-866"> } catch (Exception e) {</span> |
| <span class="source-line-no">867</span><span id="line-867"> LOG.error("Row " + Bytes.toStringBinary(row) + ": exception", e);</span> |
| <span class="source-line-no">868</span><span id="line-868"> output.getCounter(Counts.UNREFERENCED).increment(1);</span> |
| <span class="source-line-no">869</span><span id="line-869"> return;</span> |
| <span class="source-line-no">870</span><span id="line-870"> }</span> |
| <span class="source-line-no">871</span><span id="line-871"> final byte[] crcBytes = r.getValue(INFO_FAMILY_NAME, CRC_QUALIFIER);</span> |
| <span class="source-line-no">872</span><span id="line-872"> if (crcBytes == null) {</span> |
| <span class="source-line-no">873</span><span id="line-873"> LOG.error("Row " + Bytes.toStringBinary(row) + ": missing i:c");</span> |
| <span class="source-line-no">874</span><span id="line-874"> output.getCounter(Counts.UNREFERENCED).increment(1);</span> |
| <span class="source-line-no">875</span><span id="line-875"> return;</span> |
| <span class="source-line-no">876</span><span id="line-876"> }</span> |
| <span class="source-line-no">877</span><span id="line-877"> if (Bytes.toLong(crcBytes) != expectedCRC64) {</span> |
| <span class="source-line-no">878</span><span id="line-878"> LOG.error("Row " + Bytes.toStringBinary(row) + ": i:c mismatch");</span> |
| <span class="source-line-no">879</span><span id="line-879"> output.getCounter(Counts.CORRUPT).increment(1);</span> |
| <span class="source-line-no">880</span><span id="line-880"> return;</span> |
| <span class="source-line-no">881</span><span id="line-881"> }</span> |
| <span class="source-line-no">882</span><span id="line-882"> // If we fell through to here all verification checks have succeeded for the info</span> |
| <span class="source-line-no">883</span><span id="line-883"> // record.</span> |
| <span class="source-line-no">884</span><span id="line-884"> output.getCounter(Counts.REFERENCED).increment(1);</span> |
| <span class="source-line-no">885</span><span id="line-885"> final byte[] content = r.getValue(CONTENT_FAMILY_NAME, CONTENT_QUALIFIER);</span> |
| <span class="source-line-no">886</span><span id="line-886"> if (content == null) {</span> |
| <span class="source-line-no">887</span><span id="line-887"> LOG.error("Row " + Bytes.toStringBinary(row) + ": missing content");</span> |
| <span class="source-line-no">888</span><span id="line-888"> output.getCounter(Counts.UNREFERENCED).increment(1);</span> |
| <span class="source-line-no">889</span><span id="line-889"> return;</span> |
| <span class="source-line-no">890</span><span id="line-890"> } else {</span> |
| <span class="source-line-no">891</span><span id="line-891"> final CRC64 crc = new CRC64();</span> |
| <span class="source-line-no">892</span><span id="line-892"> crc.update(content);</span> |
| <span class="source-line-no">893</span><span id="line-893"> if (crc.getValue() != expectedCRC64) {</span> |
| <span class="source-line-no">894</span><span id="line-894"> LOG.error("Row " + Bytes.toStringBinary(row) + ": corrupt content");</span> |
| <span class="source-line-no">895</span><span id="line-895"> output.getCounter(Counts.CORRUPT).increment(1);</span> |
| <span class="source-line-no">896</span><span id="line-896"> return;</span> |
| <span class="source-line-no">897</span><span id="line-897"> }</span> |
| <span class="source-line-no">898</span><span id="line-898"> }</span> |
| <span class="source-line-no">899</span><span id="line-899"> // If we fell through to here all verification checks have succeeded for the content</span> |
| <span class="source-line-no">900</span><span id="line-900"> // record.</span> |
| <span class="source-line-no">901</span><span id="line-901"> output.getCounter(Counts.REFERENCED).increment(1);</span> |
| <span class="source-line-no">902</span><span id="line-902"> } else {</span> |
| <span class="source-line-no">903</span><span id="line-903"> final long startTime = System.currentTimeMillis();</span> |
| <span class="source-line-no">904</span><span id="line-904"> final Get get = new Get(row).setTimestamp(ts).addColumn(family, qualifier);</span> |
| <span class="source-line-no">905</span><span id="line-905"> Result r;</span> |
| <span class="source-line-no">906</span><span id="line-906"> try {</span> |
| <span class="source-line-no">907</span><span id="line-907"> r = table.get(get);</span> |
| <span class="source-line-no">908</span><span id="line-908"> output.getCounter(Counts.RPC_TIME_MS).increment(System.currentTimeMillis() - startTime);</span> |
| <span class="source-line-no">909</span><span id="line-909"> } catch (Exception e) {</span> |
| <span class="source-line-no">910</span><span id="line-910"> LOG.error("Row " + Bytes.toStringBinary(row) + ": exception", e);</span> |
| <span class="source-line-no">911</span><span id="line-911"> output.getCounter(Counts.UNREFERENCED).increment(1);</span> |
| <span class="source-line-no">912</span><span id="line-912"> return;</span> |
| <span class="source-line-no">913</span><span id="line-913"> }</span> |
| <span class="source-line-no">914</span><span id="line-914"> final byte[] bytes = r.getValue(family, qualifier);</span> |
| <span class="source-line-no">915</span><span id="line-915"> if (bytes == null) {</span> |
| <span class="source-line-no">916</span><span id="line-916"> LOG.error("Row " + Bytes.toStringBinary(row) + ": missing "</span> |
| <span class="source-line-no">917</span><span id="line-917"> + Bytes.toStringBinary(family) + ":" + Bytes.toStringBinary(qualifier));</span> |
| <span class="source-line-no">918</span><span id="line-918"> output.getCounter(Counts.UNREFERENCED).increment(1);</span> |
| <span class="source-line-no">919</span><span id="line-919"> return;</span> |
| <span class="source-line-no">920</span><span id="line-920"> }</span> |
| <span class="source-line-no">921</span><span id="line-921"> if (!Bytes.equals(bytes, 0, bytes.length, value.getBytes(), 0, value.getLength())) {</span> |
| <span class="source-line-no">922</span><span id="line-922"> LOG.error("Row " + Bytes.toStringBinary(row) + ": " + Bytes.toStringBinary(family) + ":"</span> |
| <span class="source-line-no">923</span><span id="line-923"> + Bytes.toStringBinary(qualifier) + " mismatch");</span> |
| <span class="source-line-no">924</span><span id="line-924"> output.getCounter(Counts.CORRUPT).increment(1);</span> |
| <span class="source-line-no">925</span><span id="line-925"> return;</span> |
| <span class="source-line-no">926</span><span id="line-926"> }</span> |
| <span class="source-line-no">927</span><span id="line-927"> // If we fell through to here all verification checks have succeeded for the info</span> |
| <span class="source-line-no">928</span><span id="line-928"> // record.</span> |
| <span class="source-line-no">929</span><span id="line-929"> output.getCounter(Counts.REFERENCED).increment(1);</span> |
| <span class="source-line-no">930</span><span id="line-930"> }</span> |
| <span class="source-line-no">931</span><span id="line-931"> }</span> |
| <span class="source-line-no">932</span><span id="line-932"> }</span> |
| <span class="source-line-no">933</span><span id="line-933"> }</span> |
| <span class="source-line-no">934</span><span id="line-934"></span> |
| <span class="source-line-no">935</span><span id="line-935"> private static final AtomicLong counter = new AtomicLong();</span> |
| <span class="source-line-no">936</span><span id="line-936"> private static final int shift = 8;</span> |
| <span class="source-line-no">937</span><span id="line-937"></span> |
| <span class="source-line-no">938</span><span id="line-938"> private static long getSequence() {</span> |
| <span class="source-line-no">939</span><span id="line-939"> long t = EnvironmentEdgeManager.currentTime();</span> |
| <span class="source-line-no">940</span><span id="line-940"> t <<= shift;</span> |
| <span class="source-line-no">941</span><span id="line-941"> t |= (counter.getAndIncrement() % (1 << shift));</span> |
| <span class="source-line-no">942</span><span id="line-942"> return t;</span> |
| <span class="source-line-no">943</span><span id="line-943"> }</span> |
| <span class="source-line-no">944</span><span id="line-944"></span> |
| <span class="source-line-no">945</span><span id="line-945"> private static byte[] rowKeyFromTargetURI(final String targetUri)</span> |
| <span class="source-line-no">946</span><span id="line-946"> throws IOException, URISyntaxException, IllegalArgumentException {</span> |
| <span class="source-line-no">947</span><span id="line-947"> final URI uri = new URI(targetUri);</span> |
| <span class="source-line-no">948</span><span id="line-948"> // Ignore the scheme</span> |
| <span class="source-line-no">949</span><span id="line-949"> // Reverse the components of the hostname</span> |
| <span class="source-line-no">950</span><span id="line-950"> String reversedHost;</span> |
| <span class="source-line-no">951</span><span id="line-951"> if (uri.getHost() != null) {</span> |
| <span class="source-line-no">952</span><span id="line-952"> final String[] hostComponents =</span> |
| <span class="source-line-no">953</span><span id="line-953"> Splitter.on('.').splitToStream(uri.getHost()).toArray(String[]::new);</span> |
| <span class="source-line-no">954</span><span id="line-954"> final StringBuilder sb = new StringBuilder();</span> |
| <span class="source-line-no">955</span><span id="line-955"> for (int i = hostComponents.length - 1; i >= 0; i--) {</span> |
| <span class="source-line-no">956</span><span id="line-956"> sb.append(hostComponents[i]);</span> |
| <span class="source-line-no">957</span><span id="line-957"> if (i != 0) {</span> |
| <span class="source-line-no">958</span><span id="line-958"> sb.append('.');</span> |
| <span class="source-line-no">959</span><span id="line-959"> }</span> |
| <span class="source-line-no">960</span><span id="line-960"> }</span> |
| <span class="source-line-no">961</span><span id="line-961"> reversedHost = sb.toString();</span> |
| <span class="source-line-no">962</span><span id="line-962"> } else {</span> |
| <span class="source-line-no">963</span><span id="line-963"> throw new IllegalArgumentException("URI is missing host component");</span> |
| <span class="source-line-no">964</span><span id="line-964"> }</span> |
| <span class="source-line-no">965</span><span id="line-965"> final ByteArrayOutputStream os = new ByteArrayOutputStream();</span> |
| <span class="source-line-no">966</span><span id="line-966"> os.write(reversedHost.getBytes(StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">967</span><span id="line-967"> if (uri.getPort() >= 0) {</span> |
| <span class="source-line-no">968</span><span id="line-968"> os.write(String.format(":%d", uri.getPort()).getBytes(StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">969</span><span id="line-969"> }</span> |
| <span class="source-line-no">970</span><span id="line-970"> os.write((byte) '|');</span> |
| <span class="source-line-no">971</span><span id="line-971"> if (uri.getPath() != null) {</span> |
| <span class="source-line-no">972</span><span id="line-972"> os.write(uri.getPath().getBytes(StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">973</span><span id="line-973"> }</span> |
| <span class="source-line-no">974</span><span id="line-974"> if (uri.getQuery() != null) {</span> |
| <span class="source-line-no">975</span><span id="line-975"> os.write(String.format("?%s", uri.getQuery()).getBytes(StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">976</span><span id="line-976"> }</span> |
| <span class="source-line-no">977</span><span id="line-977"> if (uri.getFragment() != null) {</span> |
| <span class="source-line-no">978</span><span id="line-978"> os.write(String.format("#%s", uri.getFragment()).getBytes(StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">979</span><span id="line-979"> }</span> |
| <span class="source-line-no">980</span><span id="line-980"> if (os.size() > HConstants.MAX_ROW_LENGTH) {</span> |
| <span class="source-line-no">981</span><span id="line-981"> throw new IllegalArgumentException(</span> |
| <span class="source-line-no">982</span><span id="line-982"> "Key would be too large (length=" + os.size() + ", limit=" + HConstants.MAX_ROW_LENGTH);</span> |
| <span class="source-line-no">983</span><span id="line-983"> }</span> |
| <span class="source-line-no">984</span><span id="line-984"> return os.toByteArray();</span> |
| <span class="source-line-no">985</span><span id="line-985"> }</span> |
| <span class="source-line-no">986</span><span id="line-986"></span> |
| <span class="source-line-no">987</span><span id="line-987"> static final Pattern URL_PATTERN = Pattern.compile(</span> |
| <span class="source-line-no">988</span><span id="line-988"> "\\b((https?|ftp|file)://|(www|ftp)\\.)" + "[\\-A-Z0-9+&@#/%?=~_|$!:,\\.;]*[A-Z0-9+&@#/%=~_|$]",</span> |
| <span class="source-line-no">989</span><span id="line-989"> Pattern.CASE_INSENSITIVE);</span> |
| <span class="source-line-no">990</span><span id="line-990"></span> |
| <span class="source-line-no">991</span><span id="line-991"> private static Collection<String> extractUrls(byte[] content) {</span> |
| <span class="source-line-no">992</span><span id="line-992"> final Set<String> list = new HashSet<>(); // uniques</span> |
| <span class="source-line-no">993</span><span id="line-993"> final Matcher m = URL_PATTERN.matcher(new String(content, StandardCharsets.UTF_8));</span> |
| <span class="source-line-no">994</span><span id="line-994"> while (m.find()) {</span> |
| <span class="source-line-no">995</span><span id="line-995"> list.add(m.group());</span> |
| <span class="source-line-no">996</span><span id="line-996"> }</span> |
| <span class="source-line-no">997</span><span id="line-997"> return list;</span> |
| <span class="source-line-no">998</span><span id="line-998"> }</span> |
| <span class="source-line-no">999</span><span id="line-999"></span> |
| <span class="source-line-no">1000</span><span id="line-1000">}</span> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </main> |
| </body> |
| </html> |