HUDI-123 Rename code packages/constants to org.apache.hudi (#830)
- Rename com.uber.hoodie to org.apache.hudi - Flag to pass com.uber.hoodie Input formats for hoodie-sync - Works with HUDI demo. - Also tested for backwards compatibility with datasets built by com.uber.hoodie packages - Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
committed by
vinoth chandar
parent
722b6be04a
commit
a4f9d7575f
@@ -0,0 +1,26 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hadoop;
|
||||
|
||||
/**
|
||||
* Temporary class to allow seamless migration of com.uber.hoodie to org.apache.hudi
|
||||
*/
|
||||
public class HoodieInputFormat extends org.apache.hudi.hadoop.HoodieInputFormat {
|
||||
|
||||
}
|
||||
@@ -0,0 +1,26 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hadoop.realtime;
|
||||
|
||||
/**
|
||||
* Temporary class to allow seamless migration of com.uber.hoodie to org.apache.hudi
|
||||
*/
|
||||
public class HoodieRealtimeInputFormat extends org.apache.hudi.hadoop.realtime.HoodieRealtimeInputFormat {
|
||||
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
public class HoodieHiveUtil {
|
||||
|
||||
public static final Logger LOG = LogManager.getLogger(HoodieHiveUtil.class);
|
||||
|
||||
public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode";
|
||||
public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp";
|
||||
public static final String HOODIE_MAX_COMMIT_PATTERN = "hoodie.%s.consume.max.commits";
|
||||
public static final String INCREMENTAL_SCAN_MODE = "INCREMENTAL";
|
||||
public static final String LATEST_SCAN_MODE = "LATEST";
|
||||
public static final String DEFAULT_SCAN_MODE = LATEST_SCAN_MODE;
|
||||
public static final int DEFAULT_MAX_COMMITS = 1;
|
||||
public static final int MAX_COMMIT_ALL = -1;
|
||||
public static final int DEFAULT_LEVELS_TO_BASEPATH = 3;
|
||||
|
||||
public static Integer readMaxCommits(JobContext job, String tableName) {
|
||||
String maxCommitName = String.format(HOODIE_MAX_COMMIT_PATTERN, tableName);
|
||||
int maxCommits = job.getConfiguration().getInt(maxCommitName, DEFAULT_MAX_COMMITS);
|
||||
if (maxCommits == MAX_COMMIT_ALL) {
|
||||
maxCommits = Integer.MAX_VALUE;
|
||||
}
|
||||
LOG.info("Read max commits - " + maxCommits);
|
||||
return maxCommits;
|
||||
}
|
||||
|
||||
public static String readStartCommitTime(JobContext job, String tableName) {
|
||||
String startCommitTimestampName = String.format(HOODIE_START_COMMIT_PATTERN, tableName);
|
||||
LOG.info("Read start commit time - " + job.getConfiguration().get(startCommitTimestampName));
|
||||
return job.getConfiguration().get(startCommitTimestampName);
|
||||
}
|
||||
|
||||
public static String readMode(JobContext job, String tableName) {
|
||||
String modePropertyName = String.format(HOODIE_CONSUME_MODE_PATTERN, tableName);
|
||||
String mode = job.getConfiguration().get(modePropertyName, DEFAULT_SCAN_MODE);
|
||||
LOG.info(modePropertyName + ": " + mode);
|
||||
return mode;
|
||||
}
|
||||
|
||||
public static Path getNthParent(Path path, int n) {
|
||||
Path parent = path;
|
||||
for (int i = 0; i < n; i++) {
|
||||
parent = parent.getParent();
|
||||
}
|
||||
return parent;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,224 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapred.Reporter;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.exception.DatasetNotFoundException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.InvalidDatasetException;
|
||||
|
||||
/**
|
||||
* HoodieInputFormat which understands the Hoodie File Structure and filters files based on the
|
||||
* Hoodie Mode. If paths that does not correspond to a hoodie dataset then they are passed in as is
|
||||
* (as what FileInputFormat.listStatus() would do). The JobConf could have paths from multipe
|
||||
* Hoodie/Non-Hoodie datasets
|
||||
*/
|
||||
@UseFileSplitsFromInputFormat
|
||||
public class HoodieInputFormat extends MapredParquetInputFormat implements Configurable {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(HoodieInputFormat.class);
|
||||
|
||||
protected Configuration conf;
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(JobConf job) throws IOException {
|
||||
// Get all the file status from FileInputFormat and then do the filter
|
||||
FileStatus[] fileStatuses = super.listStatus(job);
|
||||
Map<HoodieTableMetaClient, List<FileStatus>> groupedFileStatus = groupFileStatus(fileStatuses);
|
||||
LOG.info("Found a total of " + groupedFileStatus.size() + " groups");
|
||||
List<FileStatus> returns = new ArrayList<>();
|
||||
for (Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry : groupedFileStatus.entrySet()) {
|
||||
HoodieTableMetaClient metadata = entry.getKey();
|
||||
if (metadata == null) {
|
||||
// Add all the paths which are not hoodie specific
|
||||
returns.addAll(entry.getValue());
|
||||
continue;
|
||||
}
|
||||
|
||||
FileStatus[] statuses = entry.getValue().toArray(new FileStatus[entry.getValue().size()]);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
|
||||
}
|
||||
String tableName = metadata.getTableConfig().getTableName();
|
||||
String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
|
||||
// Get all commits, delta commits, compactions, as all of them produce a base parquet file
|
||||
// today
|
||||
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
ReadOptimizedView roView = new HoodieTableFileSystemView(metadata,
|
||||
timeline, statuses);
|
||||
|
||||
if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
|
||||
// this is of the form commitTs_partition_sequenceNumber
|
||||
String lastIncrementalTs = HoodieHiveUtil
|
||||
.readStartCommitTime(Job.getInstance(job), tableName);
|
||||
// Total number of commits to return in this batch. Set this to -1 to get all the commits.
|
||||
Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName);
|
||||
LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs);
|
||||
List<String> commitsToReturn = timeline.findInstantsAfter(lastIncrementalTs, maxCommits)
|
||||
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
List<HoodieDataFile> filteredFiles = roView.getLatestDataFilesInRange(commitsToReturn)
|
||||
.collect(Collectors.toList());
|
||||
for (HoodieDataFile filteredFile : filteredFiles) {
|
||||
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
|
||||
filteredFile = checkFileStatus(filteredFile);
|
||||
returns.add(filteredFile.getFileStatus());
|
||||
}
|
||||
LOG.info("Total paths to process after hoodie incremental filter " + filteredFiles.size());
|
||||
} else {
|
||||
// filter files on the latest commit found
|
||||
List<HoodieDataFile> filteredFiles = roView.getLatestDataFiles()
|
||||
.collect(Collectors.toList());
|
||||
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
|
||||
for (HoodieDataFile filteredFile : filteredFiles) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Processing latest hoodie file - " + filteredFile.getPath());
|
||||
}
|
||||
filteredFile = checkFileStatus(filteredFile);
|
||||
returns.add(filteredFile.getFileStatus());
|
||||
}
|
||||
}
|
||||
}
|
||||
return returns.toArray(new FileStatus[returns.size()]);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the file status for a race condition which can set the file size to 0. 1.
|
||||
* HiveInputFormat does super.listStatus() and gets back a FileStatus[] 2. Then it creates the
|
||||
* HoodieTableMetaClient for the paths listed. 3. Generation of splits looks at FileStatus size to
|
||||
* create splits, which skips this file
|
||||
*/
|
||||
private HoodieDataFile checkFileStatus(HoodieDataFile dataFile) throws IOException {
|
||||
Path dataPath = dataFile.getFileStatus().getPath();
|
||||
try {
|
||||
if (dataFile.getFileSize() == 0) {
|
||||
FileSystem fs = dataPath.getFileSystem(conf);
|
||||
LOG.info("Refreshing file status " + dataFile.getPath());
|
||||
return new HoodieDataFile(fs.getFileStatus(dataPath));
|
||||
}
|
||||
return dataFile;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not get FileStatus on path " + dataPath);
|
||||
}
|
||||
}
|
||||
|
||||
private Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatus(FileStatus[] fileStatuses)
|
||||
throws IOException {
|
||||
// This assumes the paths for different tables are grouped together
|
||||
Map<HoodieTableMetaClient, List<FileStatus>> grouped = new HashMap<>();
|
||||
HoodieTableMetaClient metadata = null;
|
||||
String nonHoodieBasePath = null;
|
||||
for (FileStatus status : fileStatuses) {
|
||||
if (!status.getPath().getName().endsWith(".parquet")) {
|
||||
//FIXME(vc): skip non parquet files for now. This wont be needed once log file name start
|
||||
// with "."
|
||||
continue;
|
||||
}
|
||||
if ((metadata == null && nonHoodieBasePath == null) || (metadata == null && !status.getPath()
|
||||
.toString().contains(nonHoodieBasePath)) || (metadata != null && !status.getPath()
|
||||
.toString().contains(metadata.getBasePath()))) {
|
||||
try {
|
||||
metadata = getTableMetaClient(status.getPath().getFileSystem(conf),
|
||||
status.getPath().getParent());
|
||||
nonHoodieBasePath = null;
|
||||
} catch (DatasetNotFoundException | InvalidDatasetException e) {
|
||||
LOG.info("Handling a non-hoodie path " + status.getPath());
|
||||
metadata = null;
|
||||
nonHoodieBasePath = status.getPath().getParent().toString();
|
||||
}
|
||||
if (!grouped.containsKey(metadata)) {
|
||||
grouped.put(metadata, new ArrayList<>());
|
||||
}
|
||||
}
|
||||
grouped.get(metadata).add(status);
|
||||
}
|
||||
return grouped;
|
||||
}
|
||||
|
||||
public void setConf(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
public Configuration getConf() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split,
|
||||
final JobConf job, final Reporter reporter) throws IOException {
|
||||
// TODO enable automatic predicate pushdown after fixing issues
|
||||
// FileSplit fileSplit = (FileSplit) split;
|
||||
// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent());
|
||||
// String tableName = metadata.getTableName();
|
||||
// String mode = HoodieHiveUtil.readMode(job, tableName);
|
||||
|
||||
// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
|
||||
// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split);
|
||||
// LOG.info("Setting parquet predicate push down as " + predicate);
|
||||
// ParquetInputFormat.setFilterPredicate(job, predicate);
|
||||
//clearOutExistingPredicate(job);
|
||||
// }
|
||||
return super.getRecordReader(split, job, reporter);
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the table metadata from a data path. This assumes certain hierarchy of files which should
|
||||
* be changed once a better way is figured out to pass in the hoodie meta directory
|
||||
*/
|
||||
protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path dataPath)
|
||||
throws IOException {
|
||||
int levels = HoodieHiveUtil.DEFAULT_LEVELS_TO_BASEPATH;
|
||||
if (HoodiePartitionMetadata.hasPartitionMetadata(fs, dataPath)) {
|
||||
HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, dataPath);
|
||||
metadata.readFromFS();
|
||||
levels = metadata.getPartitionDepth();
|
||||
}
|
||||
Path baseDir = HoodieHiveUtil.getNthParent(dataPath, levels);
|
||||
LOG.info("Reading hoodie metadata from path " + baseDir.toString());
|
||||
return new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,183 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.exception.DatasetNotFoundException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path -
|
||||
* Non-Hoodie dataset = then always accept
|
||||
* <p>
|
||||
* We can set this filter, on a query engine's Hadoop Config and if it respects path filters, then
|
||||
* you should be able to query both hoodie and non-hoodie datasets as you would normally do.
|
||||
* <p>
|
||||
* hadoopConf.setClass("mapreduce.input.pathFilter.class", org.apache.hudi.hadoop
|
||||
* .HoodieROTablePathFilter.class, org.apache.hadoop.fs.PathFilter.class)
|
||||
*/
|
||||
public class HoodieROTablePathFilter implements PathFilter, Serializable {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(HoodieROTablePathFilter.class);
|
||||
|
||||
/**
|
||||
* Its quite common, to have all files from a given partition path be passed into accept(), cache
|
||||
* the check for hoodie metadata for known partition paths and the latest versions of files
|
||||
*/
|
||||
private HashMap<String, HashSet<Path>> hoodiePathCache;
|
||||
|
||||
/**
|
||||
* Paths that are known to be non-hoodie datasets.
|
||||
*/
|
||||
private HashSet<String> nonHoodiePathCache;
|
||||
|
||||
|
||||
private transient FileSystem fs;
|
||||
|
||||
|
||||
public HoodieROTablePathFilter() {
|
||||
hoodiePathCache = new HashMap<>();
|
||||
nonHoodiePathCache = new HashSet<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain the path, two levels from provided path
|
||||
*
|
||||
* @return said path if available, null otherwise
|
||||
*/
|
||||
private Path safeGetParentsParent(Path path) {
|
||||
if (path.getParent() != null && path.getParent().getParent() != null
|
||||
&& path.getParent().getParent().getParent() != null) {
|
||||
return path.getParent().getParent().getParent();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean accept(Path path) {
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Checking acceptance for path " + path);
|
||||
}
|
||||
Path folder = null;
|
||||
try {
|
||||
if (fs == null) {
|
||||
fs = path.getFileSystem(new Configuration());
|
||||
}
|
||||
|
||||
// Assumes path is a file
|
||||
folder = path.getParent(); // get the immediate parent.
|
||||
// Try to use the caches.
|
||||
if (nonHoodiePathCache.contains(folder.toString())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Accepting non-hoodie path from cache: " + path);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
if (hoodiePathCache.containsKey(folder.toString())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n", path,
|
||||
hoodiePathCache.get(folder.toString()).contains(path)));
|
||||
}
|
||||
return hoodiePathCache.get(folder.toString()).contains(path);
|
||||
}
|
||||
|
||||
// Skip all files that are descendants of .hoodie in its path.
|
||||
String filePath = path.toString();
|
||||
if (filePath.contains("/" + HoodieTableMetaClient.METAFOLDER_NAME + "/")
|
||||
|| filePath.endsWith("/" + HoodieTableMetaClient.METAFOLDER_NAME)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("Skipping Hoodie Metadata file %s \n", filePath));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
// Perform actual checking.
|
||||
Path baseDir;
|
||||
if (HoodiePartitionMetadata.hasPartitionMetadata(fs, folder)) {
|
||||
HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, folder);
|
||||
metadata.readFromFS();
|
||||
baseDir = HoodieHiveUtil.getNthParent(folder, metadata.getPartitionDepth());
|
||||
} else {
|
||||
baseDir = safeGetParentsParent(folder);
|
||||
}
|
||||
|
||||
if (baseDir != null) {
|
||||
try {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(),
|
||||
baseDir.toString());
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(),
|
||||
fs.listStatus(folder));
|
||||
List<HoodieDataFile> latestFiles = fsView.getLatestDataFiles()
|
||||
.collect(Collectors.toList());
|
||||
// populate the cache
|
||||
if (!hoodiePathCache.containsKey(folder.toString())) {
|
||||
hoodiePathCache.put(folder.toString(), new HashSet<>());
|
||||
}
|
||||
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + ", caching "
|
||||
+ latestFiles.size() + " files under " + folder);
|
||||
for (HoodieDataFile lfile : latestFiles) {
|
||||
hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath()));
|
||||
}
|
||||
|
||||
// accept the path, if its among the latest files.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("%s checked after cache population, accept => %s \n", path,
|
||||
hoodiePathCache.get(folder.toString()).contains(path)));
|
||||
}
|
||||
return hoodiePathCache.get(folder.toString()).contains(path);
|
||||
} catch (DatasetNotFoundException e) {
|
||||
// Non-hoodie path, accept it.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", folder.toString()));
|
||||
}
|
||||
nonHoodiePathCache.add(folder.toString());
|
||||
return true;
|
||||
}
|
||||
} else {
|
||||
// files is at < 3 level depth in FS tree, can't be hoodie dataset
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("(2) Caching non-hoodie path under %s \n", folder.toString()));
|
||||
}
|
||||
nonHoodiePathCache.add(folder.toString());
|
||||
return true;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
String msg = "Error checking path :" + path + ", under folder: " + folder;
|
||||
LOG.error(msg, e);
|
||||
throw new HoodieException(msg, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,83 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* Provides Iterator Interface to iterate value entries read from record reader
|
||||
*
|
||||
* @param <K> Key Type
|
||||
* @param <V> Value Type
|
||||
*/
|
||||
public class RecordReaderValueIterator<K, V> implements Iterator<V> {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(RecordReaderValueIterator.class);
|
||||
|
||||
private final RecordReader<K, V> reader;
|
||||
private V nextVal = null;
|
||||
|
||||
/**
|
||||
* Construct RecordReaderValueIterator
|
||||
*
|
||||
* @param reader reader
|
||||
*/
|
||||
public RecordReaderValueIterator(RecordReader<K, V> reader) {
|
||||
this.reader = reader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
if (nextVal == null) {
|
||||
K key = reader.createKey();
|
||||
V val = reader.createValue();
|
||||
try {
|
||||
boolean notDone = reader.next(key, val);
|
||||
if (!notDone) {
|
||||
return false;
|
||||
}
|
||||
this.nextVal = val;
|
||||
} catch (IOException e) {
|
||||
LOG.error("Got error reading next record from record reader");
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public V next() {
|
||||
if (!hasNext()) {
|
||||
throw new NoSuchElementException("Make sure you are following iterator contract.");
|
||||
}
|
||||
V retVal = this.nextVal;
|
||||
this.nextVal = null;
|
||||
return retVal;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
this.reader.close();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,92 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
|
||||
/**
|
||||
* Record Reader for parquet. Records read from this reader is safe to be
|
||||
* buffered for concurrent processing.
|
||||
*
|
||||
* In concurrent producer/consumer pattern, where the record is read and buffered by one thread and processed in
|
||||
* another thread, we need to ensure new instance of ArrayWritable is buffered. ParquetReader createKey/Value is unsafe
|
||||
* as it gets reused for subsequent fetch. This wrapper makes ParquetReader safe for this use-case.
|
||||
*/
|
||||
public class SafeParquetRecordReaderWrapper implements RecordReader<NullWritable, ArrayWritable> {
|
||||
|
||||
// real Parquet reader to be wrapped
|
||||
private final RecordReader<NullWritable, ArrayWritable> parquetReader;
|
||||
|
||||
// Value Class
|
||||
private final Class valueClass;
|
||||
|
||||
// Number of fields in Value Schema
|
||||
private final int numValueFields;
|
||||
|
||||
|
||||
public SafeParquetRecordReaderWrapper(RecordReader<NullWritable, ArrayWritable> parquetReader) {
|
||||
this.parquetReader = parquetReader;
|
||||
ArrayWritable arrayWritable = parquetReader.createValue();
|
||||
this.valueClass = arrayWritable.getValueClass();
|
||||
this.numValueFields = arrayWritable.get().length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean next(NullWritable key, ArrayWritable value) throws IOException {
|
||||
return parquetReader.next(key, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NullWritable createKey() {
|
||||
return parquetReader.createKey();
|
||||
}
|
||||
|
||||
/**
|
||||
* We could be in concurrent fetch and read env.
|
||||
* We need to ensure new ArrayWritable as ParquetReader implementation reuses same
|
||||
* ArrayWritable for all reads which will cause corruption when buffering.
|
||||
* So, we create a new ArrayWritable here with Value class from parquetReader's value
|
||||
* and an empty array.
|
||||
*/
|
||||
@Override
|
||||
public ArrayWritable createValue() {
|
||||
// Call createValue of parquetReader to get size and class type info only
|
||||
Writable[] emptyWritableBuf = new Writable[numValueFields];
|
||||
return new ArrayWritable(valueClass, emptyWritableBuf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
return parquetReader.getPos();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
parquetReader.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException {
|
||||
return parquetReader.getProgress();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,38 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import java.lang.annotation.Documented;
|
||||
import java.lang.annotation.ElementType;
|
||||
import java.lang.annotation.Inherited;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
|
||||
/**
|
||||
* When annotated on a InputFormat, informs the query engines, that they should use the FileSplits
|
||||
* provided by the input format to execute the queries
|
||||
*/
|
||||
@Inherited
|
||||
@Documented
|
||||
@Target(ElementType.TYPE)
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
public @interface UseFileSplitsFromInputFormat {
|
||||
|
||||
}
|
||||
@@ -0,0 +1,933 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop.hive;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hive.common.StringInternUtils;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.ql.exec.Operator;
|
||||
import org.apache.hadoop.hive.ql.exec.Utilities;
|
||||
import org.apache.hadoop.hive.ql.io.CombineHiveRecordReader;
|
||||
import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
|
||||
import org.apache.hadoop.hive.ql.io.HiveInputFormat;
|
||||
import org.apache.hadoop.hive.ql.io.IOPrepareCache;
|
||||
import org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat;
|
||||
import org.apache.hadoop.hive.ql.log.PerfLogger;
|
||||
import org.apache.hadoop.hive.ql.parse.SplitSample;
|
||||
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
|
||||
import org.apache.hadoop.hive.ql.plan.PartitionDesc;
|
||||
import org.apache.hadoop.hive.ql.plan.TableDesc;
|
||||
import org.apache.hadoop.hive.ql.session.SessionState;
|
||||
import org.apache.hadoop.hive.shims.HadoopShims.CombineFileInputFormatShim;
|
||||
import org.apache.hadoop.hive.shims.HadoopShimsSecure;
|
||||
import org.apache.hadoop.hive.shims.HadoopShimsSecure.InputSplitShim;
|
||||
import org.apache.hadoop.hive.shims.ShimLoader;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.InputFormat;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapred.Reporter;
|
||||
import org.apache.hadoop.mapred.lib.CombineFileInputFormat;
|
||||
import org.apache.hadoop.mapred.lib.CombineFileSplit;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hudi.hadoop.HoodieInputFormat;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieRealtimeInputFormat;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* This is just a copy of the org.apache.hadoop.hive.ql.io.CombineHiveInputFormat from Hive 2.x
|
||||
* Search for **MOD** to see minor modifications to support custom inputformat in CombineHiveInputFormat.
|
||||
* See https://issues.apache.org/jira/browse/HIVE-9771
|
||||
* <p>
|
||||
* <p>
|
||||
* CombineHiveInputFormat is a parameterized InputFormat which looks at the path
|
||||
* name and determine the correct InputFormat for that path name from
|
||||
* mapredPlan.pathToPartitionInfo(). It can be used to read files with different
|
||||
* input format in the same map-reduce job.
|
||||
*
|
||||
* NOTE : This class is implemented to work with Hive 2.x +
|
||||
*/
|
||||
public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extends Writable>
|
||||
extends HiveInputFormat<K, V> {
|
||||
|
||||
private static final String CLASS_NAME = HoodieCombineHiveInputFormat.class.getName();
|
||||
public static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
|
||||
|
||||
// max number of threads we can use to check non-combinable paths
|
||||
private static final int MAX_CHECK_NONCOMBINABLE_THREAD_NUM = 50;
|
||||
private static final int DEFAULT_NUM_PATH_PER_THREAD = 100;
|
||||
|
||||
private class CheckNonCombinablePathCallable implements Callable<Set<Integer>> {
|
||||
|
||||
private final Path[] paths;
|
||||
private final int start;
|
||||
private final int length;
|
||||
private final JobConf conf;
|
||||
|
||||
public CheckNonCombinablePathCallable(Path[] paths, int start, int length, JobConf conf) {
|
||||
this.paths = paths;
|
||||
this.start = start;
|
||||
this.length = length;
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Integer> call() throws Exception {
|
||||
Set<Integer> nonCombinablePathIndices = new HashSet<Integer>();
|
||||
for (int i = 0; i < length; i++) {
|
||||
PartitionDesc part =
|
||||
HiveFileFormatUtils.getPartitionDescFromPathRecursively(
|
||||
pathToPartitionInfo, paths[i + start],
|
||||
IOPrepareCache.get().allocatePartitionDescMap());
|
||||
// Use HiveInputFormat if any of the paths is not splittable
|
||||
Class<? extends InputFormat> inputFormatClass = part.getInputFileFormatClass();
|
||||
InputFormat<WritableComparable, Writable> inputFormat =
|
||||
getInputFormatFromCache(inputFormatClass, conf);
|
||||
if (inputFormat instanceof AvoidSplitCombination
|
||||
&& ((AvoidSplitCombination) inputFormat).shouldSkipCombine(paths[i + start], conf)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("The path [" + paths[i + start]
|
||||
+ "] is being parked for HiveInputFormat.getSplits");
|
||||
}
|
||||
nonCombinablePathIndices.add(i + start);
|
||||
}
|
||||
}
|
||||
return nonCombinablePathIndices;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* CombineHiveInputSplit encapsulates an InputSplit with its corresponding
|
||||
* inputFormatClassName. A CombineHiveInputSplit comprises of multiple chunks
|
||||
* from different files. Since, they belong to a single directory, there is a
|
||||
* single inputformat for all the chunks.
|
||||
*/
|
||||
public static class CombineHiveInputSplit extends InputSplitShim {
|
||||
|
||||
private String inputFormatClassName;
|
||||
private CombineFileSplit inputSplitShim;
|
||||
private Map<Path, PartitionDesc> pathToPartitionInfo;
|
||||
|
||||
public CombineHiveInputSplit() throws IOException {
|
||||
this(ShimLoader.getHadoopShims().getCombineFileInputFormat()
|
||||
.getInputSplitShim());
|
||||
}
|
||||
|
||||
public CombineHiveInputSplit(CombineFileSplit inputSplitShim) throws IOException {
|
||||
this(inputSplitShim.getJob(), inputSplitShim);
|
||||
}
|
||||
|
||||
public CombineHiveInputSplit(JobConf job, CombineFileSplit inputSplitShim)
|
||||
throws IOException {
|
||||
this(job, inputSplitShim, null);
|
||||
}
|
||||
|
||||
public CombineHiveInputSplit(JobConf job, CombineFileSplit inputSplitShim,
|
||||
Map<Path, PartitionDesc> pathToPartitionInfo) throws IOException {
|
||||
this.inputSplitShim = inputSplitShim;
|
||||
this.pathToPartitionInfo = pathToPartitionInfo;
|
||||
if (job != null) {
|
||||
if (this.pathToPartitionInfo == null) {
|
||||
this.pathToPartitionInfo = Utilities.getMapWork(job).getPathToPartitionInfo();
|
||||
}
|
||||
|
||||
// extract all the inputFormatClass names for each chunk in the
|
||||
// CombinedSplit.
|
||||
Path[] ipaths = inputSplitShim.getPaths();
|
||||
if (ipaths.length > 0) {
|
||||
PartitionDesc part = HiveFileFormatUtils
|
||||
.getPartitionDescFromPathRecursively(this.pathToPartitionInfo,
|
||||
ipaths[0], IOPrepareCache.get().getPartitionDescMap());
|
||||
inputFormatClassName = part.getInputFileFormatClass().getName();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public CombineFileSplit getInputSplitShim() {
|
||||
return inputSplitShim;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the inputFormat class name for the i-th chunk.
|
||||
*/
|
||||
public String inputFormatClassName() {
|
||||
return inputFormatClassName;
|
||||
}
|
||||
|
||||
public void setInputFormatClassName(String inputFormatClassName) {
|
||||
this.inputFormatClassName = inputFormatClassName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JobConf getJob() {
|
||||
return inputSplitShim.getJob();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLength() {
|
||||
return inputSplitShim.getLength();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an array containing the startoffsets of the files in the split.
|
||||
*/
|
||||
@Override
|
||||
public long[] getStartOffsets() {
|
||||
return inputSplitShim.getStartOffsets();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an array containing the lengths of the files in the split.
|
||||
*/
|
||||
@Override
|
||||
public long[] getLengths() {
|
||||
return inputSplitShim.getLengths();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the start offset of the i<sup>th</sup> Path.
|
||||
*/
|
||||
@Override
|
||||
public long getOffset(int i) {
|
||||
return inputSplitShim.getOffset(i);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the length of the i<sup>th</sup> Path.
|
||||
*/
|
||||
@Override
|
||||
public long getLength(int i) {
|
||||
return inputSplitShim.getLength(i);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of Paths in the split.
|
||||
*/
|
||||
@Override
|
||||
public int getNumPaths() {
|
||||
return inputSplitShim.getNumPaths();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the i<sup>th</sup> Path.
|
||||
*/
|
||||
@Override
|
||||
public Path getPath(int i) {
|
||||
return inputSplitShim.getPath(i);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the Paths in the split.
|
||||
*/
|
||||
@Override
|
||||
public Path[] getPaths() {
|
||||
return inputSplitShim.getPaths();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the Paths where this input-split resides.
|
||||
*/
|
||||
@Override
|
||||
public String[] getLocations() throws IOException {
|
||||
return inputSplitShim.getLocations();
|
||||
}
|
||||
|
||||
/**
|
||||
* Prints this obejct as a string.
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append(inputSplitShim.toString());
|
||||
sb.append("InputFormatClass: " + inputFormatClassName);
|
||||
sb.append("\n");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Writable interface.
|
||||
*/
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
inputSplitShim.readFields(in);
|
||||
inputFormatClassName = in.readUTF();
|
||||
}
|
||||
|
||||
/**
|
||||
* Writable interface.
|
||||
*/
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
inputSplitShim.write(out);
|
||||
if (inputFormatClassName == null) {
|
||||
if (pathToPartitionInfo == null) {
|
||||
pathToPartitionInfo = Utilities.getMapWork(getJob()).getPathToPartitionInfo();
|
||||
}
|
||||
|
||||
// extract all the inputFormatClass names for each chunk in the
|
||||
// CombinedSplit.
|
||||
PartitionDesc part =
|
||||
HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo,
|
||||
inputSplitShim.getPath(0), IOPrepareCache.get().getPartitionDescMap());
|
||||
|
||||
// create a new InputFormat instance if this is the first time to see
|
||||
// this class
|
||||
inputFormatClassName = part.getInputFileFormatClass().getName();
|
||||
}
|
||||
|
||||
out.writeUTF(inputFormatClassName);
|
||||
}
|
||||
}
|
||||
|
||||
// Splits are not shared across different partitions with different input formats.
|
||||
// For example, 2 partitions (1 sequencefile and 1 rcfile) will have 2 different splits
|
||||
private static class CombinePathInputFormat {
|
||||
|
||||
private final List<Operator<? extends OperatorDesc>> opList;
|
||||
private final String inputFormatClassName;
|
||||
private final String deserializerClassName;
|
||||
|
||||
public CombinePathInputFormat(List<Operator<? extends OperatorDesc>> opList,
|
||||
String inputFormatClassName,
|
||||
String deserializerClassName) {
|
||||
this.opList = opList;
|
||||
this.inputFormatClassName = inputFormatClassName;
|
||||
this.deserializerClassName = deserializerClassName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o instanceof CombinePathInputFormat) {
|
||||
CombinePathInputFormat mObj = (CombinePathInputFormat) o;
|
||||
return (opList.equals(mObj.opList))
|
||||
&& (inputFormatClassName.equals(mObj.inputFormatClassName))
|
||||
&& (deserializerClassName == null ? (mObj.deserializerClassName == null) :
|
||||
deserializerClassName.equals(mObj.deserializerClassName));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return (opList == null) ? 0 : opList.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create Hive splits based on CombineFileSplit.
|
||||
*/
|
||||
private InputSplit[] getCombineSplits(JobConf job, int numSplits,
|
||||
Map<Path, PartitionDesc> pathToPartitionInfo)
|
||||
throws IOException {
|
||||
init(job);
|
||||
Map<Path, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
|
||||
Map<String, Operator<? extends OperatorDesc>> aliasToWork =
|
||||
mrwork.getAliasToWork();
|
||||
/** MOD - Initialize a custom combine input format shim that will call listStatus on the custom inputFormat **/
|
||||
HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim
|
||||
combine = new HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim();
|
||||
|
||||
InputSplit[] splits = null;
|
||||
if (combine == null) {
|
||||
splits = super.getSplits(job, numSplits);
|
||||
return splits;
|
||||
}
|
||||
|
||||
if (combine.getInputPathsShim(job).length == 0) {
|
||||
throw new IOException("No input paths specified in job");
|
||||
}
|
||||
ArrayList<InputSplit> result = new ArrayList<InputSplit>();
|
||||
|
||||
// combine splits only from same tables and same partitions. Do not combine splits from multiple
|
||||
// tables or multiple partitions.
|
||||
Path[] paths = StringInternUtils.internUriStringsInPathArray(combine.getInputPathsShim(job));
|
||||
|
||||
List<Path> inpDirs = new ArrayList<Path>();
|
||||
List<Path> inpFiles = new ArrayList<Path>();
|
||||
Map<CombinePathInputFormat, CombineFilter> poolMap =
|
||||
new HashMap<CombinePathInputFormat, CombineFilter>();
|
||||
Set<Path> poolSet = new HashSet<Path>();
|
||||
|
||||
for (Path path : paths) {
|
||||
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(
|
||||
pathToPartitionInfo, path, IOPrepareCache.get().allocatePartitionDescMap());
|
||||
TableDesc tableDesc = part.getTableDesc();
|
||||
if ((tableDesc != null) && tableDesc.isNonNative()) {
|
||||
return super.getSplits(job, numSplits);
|
||||
}
|
||||
|
||||
// Use HiveInputFormat if any of the paths is not splittable
|
||||
Class inputFormatClass = part.getInputFileFormatClass();
|
||||
String inputFormatClassName = inputFormatClass.getName();
|
||||
InputFormat inputFormat = getInputFormatFromCache(inputFormatClass, job);
|
||||
LOG.info("Input Format => " + inputFormatClass.getName());
|
||||
// **MOD** Set the hoodie filter in the combine
|
||||
if (inputFormatClass.getName().equals(HoodieInputFormat.class.getName())) {
|
||||
combine.setHoodieFilter(true);
|
||||
} else if (inputFormatClass.getName().equals(HoodieRealtimeInputFormat.class.getName())) {
|
||||
LOG.info("Setting hoodie filter and realtime input format");
|
||||
combine.setHoodieFilter(true);
|
||||
combine.setRealTime(true);
|
||||
}
|
||||
String deserializerClassName = null;
|
||||
try {
|
||||
deserializerClassName = part.getDeserializer(job).getClass().getName();
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
FileSystem inpFs = path.getFileSystem(job);
|
||||
|
||||
//don't combine if inputformat is a SymlinkTextInputFormat
|
||||
if (inputFormat instanceof SymlinkTextInputFormat) {
|
||||
splits = super.getSplits(job, numSplits);
|
||||
return splits;
|
||||
}
|
||||
|
||||
Path filterPath = path;
|
||||
|
||||
// Does a pool exist for this path already
|
||||
CombineFilter f = null;
|
||||
List<Operator<? extends OperatorDesc>> opList = null;
|
||||
|
||||
if (!mrwork.isMapperCannotSpanPartns()) {
|
||||
//if mapper can span partitions, make sure a splits does not contain multiple
|
||||
// opList + inputFormatClassName + deserializerClassName combination
|
||||
// This is done using the Map of CombinePathInputFormat to PathFilter
|
||||
|
||||
opList = HiveFileFormatUtils.doGetWorksFromPath(
|
||||
pathToAliases, aliasToWork, filterPath);
|
||||
CombinePathInputFormat combinePathInputFormat =
|
||||
new CombinePathInputFormat(opList, inputFormatClassName, deserializerClassName);
|
||||
f = poolMap.get(combinePathInputFormat);
|
||||
if (f == null) {
|
||||
f = new CombineFilter(filterPath);
|
||||
LOG.info("CombineHiveInputSplit creating pool for "
|
||||
+ path + "; using filter path " + filterPath);
|
||||
combine.createPool(job, f);
|
||||
poolMap.put(combinePathInputFormat, f);
|
||||
} else {
|
||||
LOG.info("CombineHiveInputSplit: pool is already created for "
|
||||
+ path + "; using filter path " + filterPath);
|
||||
f.addPath(filterPath);
|
||||
}
|
||||
} else {
|
||||
// In the case of tablesample, the input paths are pointing to files rather than directories.
|
||||
// We need to get the parent directory as the filtering path so that all files in the same
|
||||
// parent directory will be grouped into one pool but not files from different parent
|
||||
// directories. This guarantees that a split will combine all files in the same partition
|
||||
// but won't cross multiple partitions if the user has asked so.
|
||||
if (!path.getFileSystem(job).getFileStatus(path).isDir()) { // path is not directory
|
||||
filterPath = path.getParent();
|
||||
inpFiles.add(path);
|
||||
poolSet.add(filterPath);
|
||||
} else {
|
||||
inpDirs.add(path);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Processing directories
|
||||
List<CombineFileSplit> iss = new ArrayList<CombineFileSplit>();
|
||||
if (!mrwork.isMapperCannotSpanPartns()) {
|
||||
//mapper can span partitions
|
||||
//combine into as few as one split, subject to the PathFilters set
|
||||
// using combine.createPool.
|
||||
iss = Arrays.asList(combine.getSplits(job, 1));
|
||||
} else {
|
||||
for (Path path : inpDirs) {
|
||||
processPaths(job, combine, iss, path);
|
||||
}
|
||||
|
||||
if (inpFiles.size() > 0) {
|
||||
// Processing files
|
||||
for (Path filterPath : poolSet) {
|
||||
combine.createPool(job, new CombineFilter(filterPath));
|
||||
}
|
||||
processPaths(job, combine, iss, inpFiles.toArray(new Path[0]));
|
||||
}
|
||||
}
|
||||
|
||||
if (mrwork.getNameToSplitSample() != null && !mrwork.getNameToSplitSample().isEmpty()) {
|
||||
iss = sampleSplits(iss);
|
||||
}
|
||||
|
||||
for (CombineFileSplit is : iss) {
|
||||
CombineHiveInputSplit csplit = new CombineHiveInputSplit(job, is, pathToPartitionInfo);
|
||||
result.add(csplit);
|
||||
}
|
||||
|
||||
LOG.info("number of splits " + result.size());
|
||||
return result.toArray(new CombineHiveInputSplit[result.size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets all the path indices that should not be combined
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public Set<Integer> getNonCombinablePathIndices(JobConf job, Path[] paths, int numThreads)
|
||||
throws ExecutionException, InterruptedException {
|
||||
LOG.info("Total number of paths: " + paths.length
|
||||
+ ", launching " + numThreads + " threads to check non-combinable ones.");
|
||||
int numPathPerThread = (int) Math.ceil((double) paths.length / numThreads);
|
||||
|
||||
ExecutorService executor = Executors.newFixedThreadPool(numThreads);
|
||||
List<Future<Set<Integer>>> futureList = new ArrayList<Future<Set<Integer>>>(numThreads);
|
||||
try {
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
int start = i * numPathPerThread;
|
||||
int length = i != numThreads - 1 ? numPathPerThread : paths.length - start;
|
||||
futureList.add(executor.submit(
|
||||
new CheckNonCombinablePathCallable(paths, start, length, job)));
|
||||
}
|
||||
Set<Integer> nonCombinablePathIndices = new HashSet<Integer>();
|
||||
for (Future<Set<Integer>> future : futureList) {
|
||||
nonCombinablePathIndices.addAll(future.get());
|
||||
}
|
||||
return nonCombinablePathIndices;
|
||||
} finally {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create Hive splits based on CombineFileSplit.
|
||||
*/
|
||||
@Override
|
||||
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
|
||||
PerfLogger perfLogger = SessionState.getPerfLogger();
|
||||
perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.GET_SPLITS);
|
||||
init(job);
|
||||
|
||||
ArrayList<InputSplit> result = new ArrayList<InputSplit>();
|
||||
|
||||
Path[] paths = getInputPaths(job);
|
||||
|
||||
List<Path> nonCombinablePaths = new ArrayList<Path>(paths.length / 2);
|
||||
List<Path> combinablePaths = new ArrayList<Path>(paths.length / 2);
|
||||
|
||||
int numThreads = Math.min(MAX_CHECK_NONCOMBINABLE_THREAD_NUM,
|
||||
(int) Math.ceil((double) paths.length / DEFAULT_NUM_PATH_PER_THREAD));
|
||||
|
||||
// This check is necessary because for Spark branch, the result array from
|
||||
// getInputPaths() above could be empty, and therefore numThreads could be 0.
|
||||
// In that case, Executors.newFixedThreadPool will fail.
|
||||
if (numThreads > 0) {
|
||||
try {
|
||||
Set<Integer> nonCombinablePathIndices = getNonCombinablePathIndices(job, paths, numThreads);
|
||||
for (int i = 0; i < paths.length; i++) {
|
||||
if (nonCombinablePathIndices.contains(i)) {
|
||||
nonCombinablePaths.add(paths[i]);
|
||||
} else {
|
||||
combinablePaths.add(paths[i]);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Error checking non-combinable path", e);
|
||||
perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS);
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
// Store the previous value for the path specification
|
||||
String oldPaths = job.get(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("The received input paths are: [" + oldPaths
|
||||
+ "] against the property "
|
||||
+ org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR);
|
||||
}
|
||||
|
||||
// Process the normal splits
|
||||
if (nonCombinablePaths.size() > 0) {
|
||||
FileInputFormat.setInputPaths(job,
|
||||
nonCombinablePaths.toArray(new Path[nonCombinablePaths.size()]));
|
||||
InputSplit[] splits = super.getSplits(job, numSplits);
|
||||
for (InputSplit split : splits) {
|
||||
result.add(split);
|
||||
}
|
||||
}
|
||||
|
||||
// Process the combine splits
|
||||
if (combinablePaths.size() > 0) {
|
||||
FileInputFormat.setInputPaths(job,
|
||||
combinablePaths.toArray(new Path[combinablePaths.size()]));
|
||||
Map<Path, PartitionDesc> pathToPartitionInfo = this.pathToPartitionInfo != null
|
||||
? this.pathToPartitionInfo : Utilities.getMapWork(job).getPathToPartitionInfo();
|
||||
InputSplit[] splits = getCombineSplits(job, numSplits, pathToPartitionInfo);
|
||||
for (InputSplit split : splits) {
|
||||
result.add(split);
|
||||
}
|
||||
}
|
||||
|
||||
// Restore the old path information back
|
||||
// This is just to prevent incompatibilities with previous versions Hive
|
||||
// if some application depends on the original value being set.
|
||||
if (oldPaths != null) {
|
||||
job.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR, oldPaths);
|
||||
}
|
||||
|
||||
// clear work from ThreadLocal after splits generated in case of thread is reused in pool.
|
||||
Utilities.clearWorkMapForConf(job);
|
||||
|
||||
LOG.info("Number of all splits " + result.size());
|
||||
perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS);
|
||||
return result.toArray(new InputSplit[result.size()]);
|
||||
}
|
||||
|
||||
private void processPaths(JobConf job, CombineFileInputFormatShim combine,
|
||||
List<CombineFileSplit> iss, Path... path) throws IOException {
|
||||
JobConf currJob = new JobConf(job);
|
||||
FileInputFormat.setInputPaths(currJob, path);
|
||||
iss.addAll(Arrays.asList(combine.getSplits(currJob, 1)));
|
||||
}
|
||||
|
||||
/**
|
||||
* MOD - Just added this for visibility
|
||||
**/
|
||||
Path[] getInputPaths(JobConf job) throws IOException {
|
||||
Path[] dirs = FileInputFormat.getInputPaths(job);
|
||||
if (dirs.length == 0) {
|
||||
// on tez we're avoiding to duplicate the file info in FileInputFormat.
|
||||
if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
|
||||
try {
|
||||
List<Path> paths = Utilities.getInputPathsTez(job, mrwork);
|
||||
dirs = paths.toArray(new Path[paths.size()]);
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Could not create input files", e);
|
||||
}
|
||||
} else {
|
||||
throw new IOException("No input paths specified in job");
|
||||
}
|
||||
}
|
||||
return dirs;
|
||||
}
|
||||
|
||||
/**
|
||||
* This function is used to sample inputs for clauses like "TABLESAMPLE(1 PERCENT)"
|
||||
* <p>
|
||||
* First, splits are grouped by alias they are for. If one split serves more than one
|
||||
* alias or not for any sampled alias, we just directly add it to returned list.
|
||||
* Then we find a list of exclusive splits for every alias to be sampled.
|
||||
* For each alias, we start from position of seedNumber%totalNumber, and keep add
|
||||
* splits until the total size hits percentage.
|
||||
*
|
||||
* @return the sampled splits
|
||||
*/
|
||||
private List<CombineFileSplit> sampleSplits(List<CombineFileSplit> splits) {
|
||||
HashMap<String, SplitSample> nameToSamples = mrwork.getNameToSplitSample();
|
||||
List<CombineFileSplit> retLists = new ArrayList<CombineFileSplit>();
|
||||
Map<String, ArrayList<CombineFileSplit>> aliasToSplitList =
|
||||
new HashMap<String, ArrayList<CombineFileSplit>>();
|
||||
Map<Path, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
|
||||
Map<Path, ArrayList<String>> pathToAliasesNoScheme = removeScheme(pathToAliases);
|
||||
|
||||
// Populate list of exclusive splits for every sampled alias
|
||||
//
|
||||
for (CombineFileSplit split : splits) {
|
||||
String alias = null;
|
||||
for (Path path : split.getPaths()) {
|
||||
boolean schemeless = path.toUri().getScheme() == null;
|
||||
List<String> l = HiveFileFormatUtils.doGetAliasesFromPath(
|
||||
schemeless ? pathToAliasesNoScheme : pathToAliases, path);
|
||||
// a path for a split unqualified the split from being sampled if:
|
||||
// 1. it serves more than one alias
|
||||
// 2. the alias it serves is not sampled
|
||||
// 3. it serves different alias than another path for the same split
|
||||
if (l.size() != 1 || !nameToSamples.containsKey(l.get(0))
|
||||
|| (alias != null && l.get(0) != alias)) {
|
||||
alias = null;
|
||||
break;
|
||||
}
|
||||
alias = l.get(0);
|
||||
}
|
||||
|
||||
if (alias != null) {
|
||||
// split exclusively serves alias, which needs to be sampled
|
||||
// add it to the split list of the alias.
|
||||
if (!aliasToSplitList.containsKey(alias)) {
|
||||
aliasToSplitList.put(alias, new ArrayList<CombineFileSplit>());
|
||||
}
|
||||
aliasToSplitList.get(alias).add(split);
|
||||
} else {
|
||||
// The split doesn't exclusively serve one alias
|
||||
retLists.add(split);
|
||||
}
|
||||
}
|
||||
|
||||
// for every sampled alias, we figure out splits to be sampled and add
|
||||
// them to return list
|
||||
//
|
||||
for (Map.Entry<String, ArrayList<CombineFileSplit>> entry : aliasToSplitList.entrySet()) {
|
||||
ArrayList<CombineFileSplit> splitList = entry.getValue();
|
||||
long totalSize = 0;
|
||||
for (CombineFileSplit split : splitList) {
|
||||
totalSize += split.getLength();
|
||||
}
|
||||
|
||||
SplitSample splitSample = nameToSamples.get(entry.getKey());
|
||||
|
||||
long targetSize = splitSample.getTargetSize(totalSize);
|
||||
int startIndex = splitSample.getSeedNum() % splitList.size();
|
||||
long size = 0;
|
||||
for (int i = 0; i < splitList.size(); i++) {
|
||||
CombineFileSplit split = splitList.get((startIndex + i) % splitList.size());
|
||||
retLists.add(split);
|
||||
long splitgLength = split.getLength();
|
||||
if (size + splitgLength >= targetSize) {
|
||||
LOG.info("Sample alias " + entry.getValue() + " using " + (i + 1) + "splits");
|
||||
if (size + splitgLength > targetSize) {
|
||||
((InputSplitShim) split).shrinkSplit(targetSize - size);
|
||||
}
|
||||
break;
|
||||
}
|
||||
size += splitgLength;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return retLists;
|
||||
}
|
||||
|
||||
Map<Path, ArrayList<String>> removeScheme(Map<Path, ArrayList<String>> pathToAliases) {
|
||||
Map<Path, ArrayList<String>> result = new HashMap<>();
|
||||
for (Map.Entry<Path, ArrayList<String>> entry : pathToAliases.entrySet()) {
|
||||
Path newKey = Path.getPathWithoutSchemeAndAuthority(entry.getKey());
|
||||
StringInternUtils.internUriStringsInPath(newKey);
|
||||
result.put(newKey, entry.getValue());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a generic Hive RecordReader than can iterate over all chunks in a
|
||||
* CombinedFileSplit.
|
||||
*/
|
||||
@Override
|
||||
public RecordReader getRecordReader(InputSplit split, JobConf job,
|
||||
Reporter reporter) throws IOException {
|
||||
if (!(split instanceof CombineHiveInputSplit)) {
|
||||
return super.getRecordReader(split, job, reporter);
|
||||
}
|
||||
|
||||
CombineHiveInputSplit hsplit = (CombineHiveInputSplit) split;
|
||||
|
||||
String inputFormatClassName = null;
|
||||
Class inputFormatClass = null;
|
||||
try {
|
||||
inputFormatClassName = hsplit.inputFormatClassName();
|
||||
inputFormatClass = job.getClassByName(inputFormatClassName);
|
||||
} catch (Exception e) {
|
||||
throw new IOException("cannot find class " + inputFormatClassName);
|
||||
}
|
||||
|
||||
pushProjectionsAndFilters(job, inputFormatClass, hsplit.getPath(0));
|
||||
|
||||
return ShimLoader.getHadoopShims().getCombineFileInputFormat()
|
||||
.getRecordReader(job,
|
||||
(CombineFileSplit) split, reporter,
|
||||
CombineHiveRecordReader.class);
|
||||
}
|
||||
|
||||
static class CombineFilter implements PathFilter {
|
||||
|
||||
private final Set<String> pStrings = new HashSet<String>();
|
||||
|
||||
// store a path prefix in this TestFilter
|
||||
// PRECONDITION: p should always be a directory
|
||||
public CombineFilter(Path p) {
|
||||
// we need to keep the path part only because the Hadoop CombineFileInputFormat will
|
||||
// pass the path part only to accept().
|
||||
// Trailing the path with a separator to prevent partial matching.
|
||||
addPath(p);
|
||||
}
|
||||
|
||||
public void addPath(Path p) {
|
||||
String pString = p.toUri().getPath();
|
||||
pStrings.add(pString);
|
||||
}
|
||||
|
||||
// returns true if the specified path matches the prefix stored
|
||||
// in this TestFilter.
|
||||
@Override
|
||||
public boolean accept(Path path) {
|
||||
boolean find = false;
|
||||
while (path != null && !find) {
|
||||
if (pStrings.contains(path.toUri().getPath())) {
|
||||
find = true;
|
||||
break;
|
||||
}
|
||||
path = path.getParent();
|
||||
}
|
||||
return find;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder s = new StringBuilder();
|
||||
s.append("PathFilter: ");
|
||||
for (String pString : pStrings) {
|
||||
s.append(pString + " ");
|
||||
}
|
||||
return s.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is a marker interface that is used to identify the formats where
|
||||
* combine split generation is not applicable
|
||||
*/
|
||||
public interface AvoidSplitCombination {
|
||||
|
||||
boolean shouldSkipCombine(Path path, Configuration conf) throws IOException;
|
||||
}
|
||||
|
||||
/**
|
||||
* **MOD** this is the implementation of CombineFileInputFormat which is a copy of
|
||||
* org.apache.hadoop.hive.shims.HadoopShimsSecure.CombineFileInputFormatShim
|
||||
* with changes in listStatus
|
||||
*/
|
||||
public static class HoodieCombineFileInputFormatShim<K, V> extends CombineFileInputFormat<K, V>
|
||||
implements org.apache.hadoop.hive.shims.HadoopShims.CombineFileInputFormatShim<K, V> {
|
||||
|
||||
private boolean hoodieFilter = false;
|
||||
private boolean isRealTime = false;
|
||||
|
||||
public HoodieCombineFileInputFormatShim() {
|
||||
}
|
||||
|
||||
public Path[] getInputPathsShim(JobConf conf) {
|
||||
try {
|
||||
return FileInputFormat.getInputPaths(conf);
|
||||
} catch (Exception var3) {
|
||||
throw new RuntimeException(var3);
|
||||
}
|
||||
}
|
||||
|
||||
public void createPool(JobConf conf, PathFilter... filters) {
|
||||
super.createPool(conf, filters);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader<K, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
|
||||
throws IOException {
|
||||
throw new IOException("CombineFileInputFormat.getRecordReader not needed.");
|
||||
}
|
||||
|
||||
protected List<FileStatus> listStatus(JobContext job) throws IOException {
|
||||
LOG.info("Listing status in HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim");
|
||||
List<FileStatus> result;
|
||||
if (hoodieFilter) {
|
||||
HoodieInputFormat input;
|
||||
if (isRealTime) {
|
||||
LOG.info("Using HoodieRealtimeInputFormat");
|
||||
input = new HoodieRealtimeInputFormat();
|
||||
} else {
|
||||
LOG.info("Using HoodieInputFormat");
|
||||
input = new HoodieInputFormat();
|
||||
}
|
||||
input.setConf(job.getConfiguration());
|
||||
result = new ArrayList<FileStatus>(
|
||||
Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
|
||||
} else {
|
||||
result = super.listStatus(job);
|
||||
}
|
||||
|
||||
Iterator it = result.iterator();
|
||||
|
||||
while (it.hasNext()) {
|
||||
FileStatus stat = (FileStatus) it.next();
|
||||
if (!stat.isFile()) {
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public CombineFileSplit[] getSplits(JobConf job, int numSplits) throws IOException {
|
||||
long minSize = job.getLong(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.SPLIT_MINSIZE, 0L);
|
||||
if (job.getLong("mapreduce.input.fileinputformat.split.minsize.per.node", 0L) == 0L) {
|
||||
super.setMinSplitSizeNode(minSize);
|
||||
}
|
||||
|
||||
if (job.getLong("mapreduce.input.fileinputformat.split.minsize.per.rack", 0L) == 0L) {
|
||||
super.setMinSplitSizeRack(minSize);
|
||||
}
|
||||
|
||||
if (job.getLong(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.SPLIT_MAXSIZE, 0L) == 0L) {
|
||||
super.setMaxSplitSize(minSize);
|
||||
}
|
||||
|
||||
InputSplit[] splits = super.getSplits(job, numSplits);
|
||||
ArrayList inputSplitShims = new ArrayList();
|
||||
|
||||
for (int pos = 0; pos < splits.length; ++pos) {
|
||||
CombineFileSplit split = (CombineFileSplit) splits[pos];
|
||||
if (split.getPaths().length > 0) {
|
||||
inputSplitShims.add(
|
||||
new HadoopShimsSecure.InputSplitShim(job, split.getPaths(), split.getStartOffsets(),
|
||||
split.getLengths(), split.getLocations()));
|
||||
}
|
||||
}
|
||||
|
||||
return (CombineFileSplit[]) inputSplitShims
|
||||
.toArray(new HadoopShimsSecure.InputSplitShim[inputSplitShims.size()]);
|
||||
}
|
||||
|
||||
public HadoopShimsSecure.InputSplitShim getInputSplitShim() throws IOException {
|
||||
return new HadoopShimsSecure.InputSplitShim();
|
||||
}
|
||||
|
||||
public RecordReader getRecordReader(JobConf job, CombineFileSplit split, Reporter reporter,
|
||||
Class<RecordReader<K, V>> rrClass) throws IOException {
|
||||
return new HadoopShimsSecure.CombineFileRecordReader(job, split, reporter, rrClass);
|
||||
}
|
||||
|
||||
public void setHoodieFilter(boolean hoodieFilter) {
|
||||
this.hoodieFilter = hoodieFilter;
|
||||
}
|
||||
|
||||
public void setRealTime(boolean realTime) {
|
||||
isRealTime = realTime;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,359 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.GenericArray;
|
||||
import org.apache.avro.generic.GenericFixed;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.hive.serde2.io.DoubleWritable;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.BooleanWritable;
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.LogReaderUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import parquet.avro.AvroSchemaConverter;
|
||||
import parquet.hadoop.ParquetFileReader;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
/**
|
||||
* Record Reader implementation to merge fresh avro data with base parquet data, to support real
|
||||
* time queries.
|
||||
*/
|
||||
public abstract class AbstractRealtimeRecordReader {
|
||||
|
||||
// Fraction of mapper/reducer task memory used for compaction of log files
|
||||
public static final String COMPACTION_MEMORY_FRACTION_PROP = "compaction.memory.fraction";
|
||||
public static final String DEFAULT_COMPACTION_MEMORY_FRACTION = "0.75";
|
||||
// used to choose a trade off between IO vs Memory when performing compaction process
|
||||
// Depending on outputfile size and memory provided, choose true to avoid OOM for large file
|
||||
// size + small memory
|
||||
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP =
|
||||
"compaction.lazy.block.read.enabled";
|
||||
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "true";
|
||||
|
||||
// Property to set the max memory for dfs inputstream buffer size
|
||||
public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size";
|
||||
// Setting this to lower value of 1 MB since no control over how many RecordReaders will be started in a mapper
|
||||
public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1 * 1024 * 1024; // 1 MB
|
||||
// Property to set file path prefix for spillable file
|
||||
public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path";
|
||||
// Default file path prefix for spillable file
|
||||
public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/";
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(AbstractRealtimeRecordReader.class);
|
||||
protected final HoodieRealtimeFileSplit split;
|
||||
protected final JobConf jobConf;
|
||||
private final MessageType baseFileSchema;
|
||||
protected final boolean usesCustomPayload;
|
||||
// Schema handles
|
||||
private Schema readerSchema;
|
||||
private Schema writerSchema;
|
||||
|
||||
public AbstractRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job) {
|
||||
this.split = split;
|
||||
this.jobConf = job;
|
||||
LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
|
||||
LOG.info("columnIds ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
|
||||
LOG.info("partitioningColumns ==> " + job.get("partition_columns", ""));
|
||||
try {
|
||||
this.usesCustomPayload = usesCustomPayload();
|
||||
LOG.info("usesCustomPayload ==> " + this.usesCustomPayload);
|
||||
baseFileSchema = readSchema(jobConf, split.getPath());
|
||||
init();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean usesCustomPayload() {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jobConf, split.getBasePath());
|
||||
return !(metaClient.getTableConfig().getPayloadClass().contains(HoodieAvroPayload.class.getName())
|
||||
|| metaClient.getTableConfig().getPayloadClass().contains("org.apache.hudi.OverwriteWithLatestAvroPayload"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the schema from the parquet file. This is different from ParquetUtils as it uses the
|
||||
* twitter parquet to support hive 1.1.0
|
||||
*/
|
||||
private static MessageType readSchema(Configuration conf, Path parquetFilePath) {
|
||||
try {
|
||||
return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Prints a JSON representation of the ArrayWritable for easier debuggability
|
||||
*/
|
||||
protected static String arrayWritableToString(ArrayWritable writable) {
|
||||
if (writable == null) {
|
||||
return "null";
|
||||
}
|
||||
StringBuilder builder = new StringBuilder();
|
||||
Writable[] values = writable.get();
|
||||
builder.append("\"values_" + Math.random() + "_" + values.length + "\": {");
|
||||
int i = 0;
|
||||
for (Writable w : values) {
|
||||
if (w instanceof ArrayWritable) {
|
||||
builder.append(arrayWritableToString((ArrayWritable) w)).append(",");
|
||||
} else {
|
||||
builder.append("\"value" + i + "\":" + "\"" + w + "\"").append(",");
|
||||
if (w == null) {
|
||||
builder.append("\"type" + i + "\":" + "\"unknown\"").append(",");
|
||||
} else {
|
||||
builder.append("\"type" + i + "\":" + "\"" + w.getClass().getSimpleName() + "\"").append(",");
|
||||
}
|
||||
}
|
||||
i++;
|
||||
}
|
||||
builder.deleteCharAt(builder.length() - 1);
|
||||
builder.append("}");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a comma separated list of field names and positions at which they appear on Hive, return
|
||||
* a ordered list of field names, that can be passed onto storage.
|
||||
*/
|
||||
private static List<String> orderFields(String fieldNameCsv, String fieldOrderCsv, List<String> partitioningFields) {
|
||||
|
||||
String[] fieldOrders = fieldOrderCsv.split(",");
|
||||
List<String> fieldNames = Arrays.stream(fieldNameCsv.split(","))
|
||||
.filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList());
|
||||
|
||||
// Hive does not provide ids for partitioning fields, so check for lengths excluding that.
|
||||
if (fieldNames.size() != fieldOrders.length) {
|
||||
throw new HoodieException(String
|
||||
.format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d",
|
||||
fieldNames.size(), fieldOrders.length));
|
||||
}
|
||||
TreeMap<Integer, String> orderedFieldMap = new TreeMap<>();
|
||||
for (int ox = 0; ox < fieldOrders.length; ox++) {
|
||||
orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNames.get(ox));
|
||||
}
|
||||
return new ArrayList<>(orderedFieldMap.values());
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a reader schema off the provided writeSchema, to just project out the provided
|
||||
* columns
|
||||
*/
|
||||
public static Schema generateProjectionSchema(Schema writeSchema, List<String> fieldNames) {
|
||||
/**
|
||||
* Avro & Presto field names seems to be case sensitive (support fields differing only in case)
|
||||
* whereas Hive/Impala/SparkSQL(default) are case-insensitive. Spark allows this to be configurable
|
||||
* using spark.sql.caseSensitive=true
|
||||
*
|
||||
* For a RT table setup with no delta-files (for a latest file-slice) -> we translate parquet schema to Avro
|
||||
* Here the field-name case is dependent on parquet schema. Hive (1.x/2.x/CDH) translate column projections
|
||||
* to lower-cases
|
||||
*
|
||||
*/
|
||||
List<Schema.Field> projectedFields = new ArrayList<>();
|
||||
Map<String, Schema.Field> schemaFieldsMap = writeSchema.getFields().stream()
|
||||
.map(r -> Pair.of(r.name().toLowerCase(), r)).collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
|
||||
for (String fn : fieldNames) {
|
||||
Schema.Field field = schemaFieldsMap.get(fn.toLowerCase());
|
||||
if (field == null) {
|
||||
throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! "
|
||||
+ "Derived Schema Fields: "
|
||||
+ new ArrayList<>(schemaFieldsMap.keySet()));
|
||||
} else {
|
||||
projectedFields
|
||||
.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue()));
|
||||
}
|
||||
}
|
||||
|
||||
Schema projectedSchema = Schema
|
||||
.createRecord(writeSchema.getName(), writeSchema.getDoc(), writeSchema.getNamespace(), writeSchema.isError());
|
||||
projectedSchema.setFields(projectedFields);
|
||||
return projectedSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert the projected read from delta record into an array writable
|
||||
*/
|
||||
public static Writable avroToArrayWritable(Object value, Schema schema) {
|
||||
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
switch (schema.getType()) {
|
||||
case STRING:
|
||||
return new Text(value.toString());
|
||||
case BYTES:
|
||||
return new BytesWritable((byte[]) value);
|
||||
case INT:
|
||||
return new IntWritable((Integer) value);
|
||||
case LONG:
|
||||
return new LongWritable((Long) value);
|
||||
case FLOAT:
|
||||
return new FloatWritable((Float) value);
|
||||
case DOUBLE:
|
||||
return new DoubleWritable((Double) value);
|
||||
case BOOLEAN:
|
||||
return new BooleanWritable((Boolean) value);
|
||||
case NULL:
|
||||
return null;
|
||||
case RECORD:
|
||||
GenericRecord record = (GenericRecord) value;
|
||||
Writable[] recordValues = new Writable[schema.getFields().size()];
|
||||
int recordValueIndex = 0;
|
||||
for (Schema.Field field : schema.getFields()) {
|
||||
recordValues[recordValueIndex++] = avroToArrayWritable(record.get(field.name()), field.schema());
|
||||
}
|
||||
return new ArrayWritable(Writable.class, recordValues);
|
||||
case ENUM:
|
||||
return new Text(value.toString());
|
||||
case ARRAY:
|
||||
GenericArray arrayValue = (GenericArray) value;
|
||||
Writable[] arrayValues = new Writable[arrayValue.size()];
|
||||
int arrayValueIndex = 0;
|
||||
for (Object obj : arrayValue) {
|
||||
arrayValues[arrayValueIndex++] = avroToArrayWritable(obj, schema.getElementType());
|
||||
}
|
||||
// Hive 1.x will fail here, it requires values2 to be wrapped into another ArrayWritable
|
||||
return new ArrayWritable(Writable.class, arrayValues);
|
||||
case MAP:
|
||||
Map mapValue = (Map) value;
|
||||
Writable[] mapValues = new Writable[mapValue.size()];
|
||||
int mapValueIndex = 0;
|
||||
for (Object entry : mapValue.entrySet()) {
|
||||
Map.Entry mapEntry = (Map.Entry) entry;
|
||||
Writable[] nestedMapValues = new Writable[2];
|
||||
nestedMapValues[0] = new Text(mapEntry.getKey().toString());
|
||||
nestedMapValues[1] = avroToArrayWritable(mapEntry.getValue(), schema.getValueType());
|
||||
mapValues[mapValueIndex++] = new ArrayWritable(Writable.class, nestedMapValues);
|
||||
}
|
||||
// Hive 1.x will fail here, it requires values3 to be wrapped into another ArrayWritable
|
||||
return new ArrayWritable(Writable.class, mapValues);
|
||||
case UNION:
|
||||
List<Schema> types = schema.getTypes();
|
||||
if (types.size() != 2) {
|
||||
throw new IllegalArgumentException("Only support union with 2 fields");
|
||||
}
|
||||
Schema s1 = types.get(0);
|
||||
Schema s2 = types.get(1);
|
||||
if (s1.getType() == Schema.Type.NULL) {
|
||||
return avroToArrayWritable(value, s2);
|
||||
} else if (s2.getType() == Schema.Type.NULL) {
|
||||
return avroToArrayWritable(value, s1);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Only support union with null");
|
||||
}
|
||||
case FIXED:
|
||||
return new BytesWritable(((GenericFixed) value).bytes());
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Hive implementation of ParquetRecordReader results in partition columns not present in the original parquet file
|
||||
* to also be part of the projected schema. Hive expects the record reader implementation to return the row in its
|
||||
* entirety (with un-projected column having null values). As we use writerSchema for this, make sure writer schema
|
||||
* also includes partition columns
|
||||
*
|
||||
* @param schema Schema to be changed
|
||||
*/
|
||||
private static Schema addPartitionFields(Schema schema, List<String> partitioningFields) {
|
||||
final Set<String> firstLevelFieldNames = schema.getFields().stream().map(Field::name)
|
||||
.map(String::toLowerCase).collect(Collectors.toSet());
|
||||
List<String> fieldsToAdd = partitioningFields.stream().map(String::toLowerCase)
|
||||
.filter(x -> !firstLevelFieldNames.contains(x)).collect(Collectors.toList());
|
||||
|
||||
return HoodieAvroUtils.appendNullSchemaFields(schema, fieldsToAdd);
|
||||
}
|
||||
|
||||
/**
|
||||
* Goes through the log files in reverse order and finds the schema from the last available data block. If not, falls
|
||||
* back to the schema from the latest parquet file. Finally, sets the partition column and projection fields into
|
||||
* the job conf.
|
||||
*/
|
||||
private void init() throws IOException {
|
||||
Schema schemaFromLogFile = LogReaderUtils
|
||||
.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaFilePaths(), jobConf);
|
||||
if (schemaFromLogFile == null) {
|
||||
writerSchema = new AvroSchemaConverter().convert(baseFileSchema);
|
||||
LOG.debug("Writer Schema From Parquet => " + writerSchema.getFields());
|
||||
} else {
|
||||
writerSchema = schemaFromLogFile;
|
||||
LOG.debug("Writer Schema From Log => " + writerSchema.getFields());
|
||||
}
|
||||
// Add partitioning fields to writer schema for resulting row to contain null values for these fields
|
||||
String partitionFields = jobConf.get("partition_columns", "");
|
||||
List<String> partitioningFields =
|
||||
partitionFields.length() > 0 ? Arrays.stream(partitionFields.split(",")).collect(Collectors.toList())
|
||||
: new ArrayList<>();
|
||||
writerSchema = addPartitionFields(writerSchema, partitioningFields);
|
||||
List<String> projectionFields = orderFields(
|
||||
jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
|
||||
jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR),
|
||||
partitioningFields);
|
||||
// TODO(vc): In the future, the reader schema should be updated based on log files & be able
|
||||
// to null out fields not present before
|
||||
readerSchema = generateProjectionSchema(writerSchema, projectionFields);
|
||||
LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s",
|
||||
split.getDeltaFilePaths(), split.getPath(), projectionFields));
|
||||
}
|
||||
|
||||
public Schema getReaderSchema() {
|
||||
return readerSchema;
|
||||
}
|
||||
|
||||
public Schema getWriterSchema() {
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
public long getMaxCompactionMemoryInBytes() {
|
||||
// jobConf.getMemoryForMapTask() returns in MB
|
||||
return (long) Math.ceil(Double
|
||||
.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
|
||||
* jobConf.getMemoryForMapTask() * 1024 * 1024L);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
|
||||
|
||||
/**
|
||||
* Simply extends ParquetHiveSerDe
|
||||
*/
|
||||
public class HoodieParquetSerde extends ParquetHiveSerDe {
|
||||
|
||||
public HoodieParquetSerde() {
|
||||
super();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
|
||||
/**
|
||||
* Filesplit that wraps the base split and a list of log files to merge deltas from.
|
||||
*/
|
||||
public class HoodieRealtimeFileSplit extends FileSplit {
|
||||
|
||||
private List<String> deltaFilePaths;
|
||||
|
||||
private String maxCommitTime;
|
||||
|
||||
private String basePath;
|
||||
|
||||
public HoodieRealtimeFileSplit() {
|
||||
super();
|
||||
}
|
||||
|
||||
public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List<String> deltaLogFiles,
|
||||
String maxCommitTime) throws IOException {
|
||||
super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(),
|
||||
baseSplit.getLocations());
|
||||
this.deltaFilePaths = deltaLogFiles;
|
||||
this.maxCommitTime = maxCommitTime;
|
||||
this.basePath = basePath;
|
||||
}
|
||||
|
||||
public List<String> getDeltaFilePaths() {
|
||||
return deltaFilePaths;
|
||||
}
|
||||
|
||||
public String getMaxCommitTime() {
|
||||
return maxCommitTime;
|
||||
}
|
||||
|
||||
public String getBasePath() {
|
||||
return basePath;
|
||||
}
|
||||
|
||||
private static void writeString(String str, DataOutput out) throws IOException {
|
||||
byte[] bytes = str.getBytes(StandardCharsets.UTF_8);
|
||||
out.writeInt(bytes.length);
|
||||
out.write(bytes);
|
||||
}
|
||||
|
||||
private static String readString(DataInput in) throws IOException {
|
||||
byte[] bytes = new byte[in.readInt()];
|
||||
in.readFully(bytes);
|
||||
return new String(bytes, StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
super.write(out);
|
||||
writeString(basePath, out);
|
||||
writeString(maxCommitTime, out);
|
||||
out.writeInt(deltaFilePaths.size());
|
||||
for (String logFilePath : deltaFilePaths) {
|
||||
writeString(logFilePath, out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
super.readFields(in);
|
||||
basePath = readString(in);
|
||||
maxCommitTime = readString(in);
|
||||
int totalLogFiles = in.readInt();
|
||||
deltaFilePaths = new ArrayList<>(totalLogFiles);
|
||||
for (int i = 0; i < totalLogFiles; i++) {
|
||||
deltaFilePaths.add(readString(in));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HoodieRealtimeFileSplit{"
|
||||
+ "DataPath=" + getPath()
|
||||
+ ", deltaFilePaths=" + deltaFilePaths
|
||||
+ ", maxCommitTime='" + maxCommitTime + '\''
|
||||
+ ", basePath='" + basePath + '\''
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,241 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapred.Reporter;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.HoodieInputFormat;
|
||||
import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat;
|
||||
|
||||
/**
|
||||
* Input Format, that provides a real-time view of data in a Hoodie dataset
|
||||
*/
|
||||
@UseFileSplitsFromInputFormat
|
||||
public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Configurable {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(HoodieRealtimeInputFormat.class);
|
||||
|
||||
// These positions have to be deterministic across all tables
|
||||
public static final int HOODIE_COMMIT_TIME_COL_POS = 0;
|
||||
public static final int HOODIE_RECORD_KEY_COL_POS = 2;
|
||||
public static final int HOODIE_PARTITION_PATH_COL_POS = 3;
|
||||
// Hive on Spark queries do not work with RT tables. Our theory is that due to
|
||||
// {@link org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher}
|
||||
// not handling empty list correctly, the ParquetRecordReaderWrapper ends up adding the same column ids multiple
|
||||
// times which ultimately breaks the query.
|
||||
|
||||
@Override
|
||||
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
|
||||
|
||||
Stream<FileSplit> fileSplits = Arrays.stream(super.getSplits(job, numSplits))
|
||||
.map(is -> (FileSplit) is);
|
||||
|
||||
// obtain all unique parent folders for splits
|
||||
Map<Path, List<FileSplit>> partitionsToParquetSplits = fileSplits
|
||||
.collect(Collectors.groupingBy(split -> split.getPath().getParent()));
|
||||
// TODO(vc): Should we handle also non-hoodie splits here?
|
||||
Map<String, HoodieTableMetaClient> metaClientMap = new HashMap<>();
|
||||
Map<Path, HoodieTableMetaClient> partitionsToMetaClient = partitionsToParquetSplits.keySet()
|
||||
.stream().collect(Collectors.toMap(Function.identity(), p -> {
|
||||
// find if we have a metaclient already for this partition.
|
||||
Option<String> matchingBasePath = Option.fromJavaOptional(metaClientMap.keySet().stream()
|
||||
.filter(basePath -> p.toString().startsWith(basePath)).findFirst());
|
||||
if (matchingBasePath.isPresent()) {
|
||||
return metaClientMap.get(matchingBasePath.get());
|
||||
}
|
||||
|
||||
try {
|
||||
HoodieTableMetaClient metaClient = getTableMetaClient(p.getFileSystem(conf), p);
|
||||
metaClientMap.put(metaClient.getBasePath(), metaClient);
|
||||
return metaClient;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Error creating hoodie meta client against : " + p, e);
|
||||
}
|
||||
}));
|
||||
|
||||
// for all unique split parents, obtain all delta files based on delta commit timeline,
|
||||
// grouped on file id
|
||||
List<HoodieRealtimeFileSplit> rtSplits = new ArrayList<>();
|
||||
partitionsToParquetSplits.keySet().stream().forEach(partitionPath -> {
|
||||
// for each partition path obtain the data & log file groupings, then map back to inputsplits
|
||||
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline());
|
||||
String relPartitionPath = FSUtils
|
||||
.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath);
|
||||
|
||||
try {
|
||||
// Both commit and delta-commits are included - pick the latest completed one
|
||||
Option<HoodieInstant> latestCompletedInstant =
|
||||
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
|
||||
Stream<FileSlice> latestFileSlices = latestCompletedInstant.map(instant ->
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp()))
|
||||
.orElse(Stream.empty());
|
||||
|
||||
// subgroup splits again by file id & match with log files.
|
||||
Map<String, List<FileSplit>> groupedInputSplits = partitionsToParquetSplits
|
||||
.get(partitionPath).stream()
|
||||
.collect(Collectors.groupingBy(split -> FSUtils.getFileId(split.getPath().getName())));
|
||||
latestFileSlices.forEach(fileSlice -> {
|
||||
List<FileSplit> dataFileSplits = groupedInputSplits.get(fileSlice.getFileId());
|
||||
dataFileSplits.forEach(split -> {
|
||||
try {
|
||||
List<String> logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(logFile -> logFile.getPath().toString()).collect(Collectors.toList());
|
||||
// Get the maxCommit from the last delta or compaction or commit - when
|
||||
// bootstrapped from COW table
|
||||
String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION)).filterCompletedInstants().lastInstant()
|
||||
.get().getTimestamp();
|
||||
rtSplits.add(
|
||||
new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths,
|
||||
maxCommitTime));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Error creating hoodie real time split ", e);
|
||||
}
|
||||
});
|
||||
});
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error obtaining data file/log file grouping: " + partitionPath,
|
||||
e);
|
||||
}
|
||||
});
|
||||
LOG.info("Returning a total splits of " + rtSplits.size());
|
||||
return rtSplits.toArray(new InputSplit[rtSplits.size()]);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(JobConf job) throws IOException {
|
||||
// Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit
|
||||
// timeline.
|
||||
return super.listStatus(job);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a field to the existing fields projected
|
||||
*/
|
||||
private static Configuration addProjectionField(Configuration conf, String fieldName,
|
||||
int fieldIndex) {
|
||||
String readColNames = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "");
|
||||
String readColIds = conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "");
|
||||
|
||||
String readColNamesPrefix = readColNames + ",";
|
||||
if (readColNames == null || readColNames.isEmpty()) {
|
||||
readColNamesPrefix = "";
|
||||
}
|
||||
String readColIdsPrefix = readColIds + ",";
|
||||
if (readColIds == null || readColIds.isEmpty()) {
|
||||
readColIdsPrefix = "";
|
||||
}
|
||||
|
||||
if (!readColNames.contains(fieldName)) {
|
||||
// If not already in the list - then add it
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, readColNamesPrefix + fieldName);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format(
|
||||
"Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ",
|
||||
conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
|
||||
conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)));
|
||||
}
|
||||
}
|
||||
return conf;
|
||||
}
|
||||
|
||||
private static synchronized Configuration addRequiredProjectionFields(Configuration configuration) {
|
||||
// Need this to do merge records in HoodieRealtimeRecordReader
|
||||
configuration = addProjectionField(configuration, HoodieRecord.RECORD_KEY_METADATA_FIELD,
|
||||
HOODIE_RECORD_KEY_COL_POS);
|
||||
configuration = addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
HOODIE_COMMIT_TIME_COL_POS);
|
||||
configuration = addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD,
|
||||
HOODIE_PARTITION_PATH_COL_POS);
|
||||
return configuration;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader<NullWritable, ArrayWritable> getRecordReader(final InputSplit split,
|
||||
final JobConf job, final Reporter reporter) throws IOException {
|
||||
|
||||
LOG.info("Before adding Hoodie columns, Projections :" + job
|
||||
.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) + ", Ids :"
|
||||
+ job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
|
||||
|
||||
// Hive (across all versions) fails for queries like select count(`_hoodie_commit_time`) from table;
|
||||
// In this case, the projection fields gets removed. Looking at HiveInputFormat implementation, in some cases
|
||||
// hoodie additional projection columns are reset after calling setConf and only natural projections
|
||||
// (one found in select queries) are set. things would break because of this.
|
||||
// For e:g _hoodie_record_key would be missing and merge step would throw exceptions.
|
||||
// TO fix this, hoodie columns are appended late at the time record-reader gets built instead of construction time.
|
||||
this.conf = addRequiredProjectionFields(job);
|
||||
|
||||
LOG.info("Creating record reader with readCols :" + job
|
||||
.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) + ", Ids :"
|
||||
+ job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
|
||||
// sanity check
|
||||
Preconditions.checkArgument(split instanceof HoodieRealtimeFileSplit,
|
||||
"HoodieRealtimeRecordReader can only work on HoodieRealtimeFileSplit and not with "
|
||||
+ split);
|
||||
|
||||
return new HoodieRealtimeRecordReader((HoodieRealtimeFileSplit) split, job,
|
||||
super.getRecordReader(split, job, reporter));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return conf;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,103 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* Realtime Record Reader which can do compacted (merge-on-read) record reading or
|
||||
* unmerged reading (parquet and log files read in parallel) based on job configuration.
|
||||
*/
|
||||
public class HoodieRealtimeRecordReader implements RecordReader<NullWritable, ArrayWritable> {
|
||||
|
||||
// Property to enable parallel reading of parquet and log files without merging.
|
||||
public static final String REALTIME_SKIP_MERGE_PROP = "hoodie.realtime.merge.skip";
|
||||
// By default, we do merged-reading
|
||||
public static final String DEFAULT_REALTIME_SKIP_MERGE = "false";
|
||||
public static final Log LOG = LogFactory.getLog(HoodieRealtimeRecordReader.class);
|
||||
private final RecordReader<NullWritable, ArrayWritable> reader;
|
||||
|
||||
public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job,
|
||||
RecordReader<NullWritable, ArrayWritable> realReader) {
|
||||
this.reader = constructRecordReader(split, job, realReader);
|
||||
}
|
||||
|
||||
public static boolean canSkipMerging(JobConf jobConf) {
|
||||
return Boolean.valueOf(jobConf.get(REALTIME_SKIP_MERGE_PROP, DEFAULT_REALTIME_SKIP_MERGE));
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct record reader based on job configuration
|
||||
*
|
||||
* @param split File Split
|
||||
* @param jobConf Job Configuration
|
||||
* @param realReader Parquet Record Reader
|
||||
* @return Realtime Reader
|
||||
*/
|
||||
private static RecordReader<NullWritable, ArrayWritable> constructRecordReader(HoodieRealtimeFileSplit split,
|
||||
JobConf jobConf, RecordReader<NullWritable, ArrayWritable> realReader) {
|
||||
try {
|
||||
if (canSkipMerging(jobConf)) {
|
||||
LOG.info("Enabling un-merged reading of realtime records");
|
||||
return new RealtimeUnmergedRecordReader(split, jobConf, realReader);
|
||||
}
|
||||
return new RealtimeCompactedRecordReader(split, jobConf, realReader);
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Got exception when constructing record reader", ex);
|
||||
throw new HoodieException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean next(NullWritable key, ArrayWritable value) throws IOException {
|
||||
return this.reader.next(key, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NullWritable createKey() {
|
||||
return this.reader.createKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayWritable createValue() {
|
||||
return this.reader.createValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
return this.reader.getPos();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
this.reader.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException {
|
||||
return this.reader.getProgress();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,147 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader implements
|
||||
RecordReader<NullWritable, ArrayWritable> {
|
||||
|
||||
protected final RecordReader<NullWritable, ArrayWritable> parquetReader;
|
||||
private final Map<String, HoodieRecord<? extends HoodieRecordPayload>> deltaRecordMap;
|
||||
|
||||
public RealtimeCompactedRecordReader(HoodieRealtimeFileSplit split, JobConf job,
|
||||
RecordReader<NullWritable, ArrayWritable> realReader) throws IOException {
|
||||
super(split, job);
|
||||
this.parquetReader = realReader;
|
||||
this.deltaRecordMap = getMergedLogRecordScanner().getRecords();
|
||||
}
|
||||
|
||||
/**
|
||||
* Goes through the log files and populates a map with latest version of each key logged, since
|
||||
* the base split was written.
|
||||
*/
|
||||
private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOException {
|
||||
// NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit
|
||||
// but can return records for completed commits > the commit we are trying to read (if using
|
||||
// readCommit() API)
|
||||
return new HoodieMergedLogRecordScanner(
|
||||
FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(),
|
||||
split.getDeltaFilePaths(), usesCustomPayload ? getWriterSchema() : getReaderSchema(), split.getMaxCommitTime(),
|
||||
getMaxCompactionMemoryInBytes(),
|
||||
Boolean.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
|
||||
DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
|
||||
jobConf.get(SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean next(NullWritable aVoid, ArrayWritable arrayWritable) throws IOException {
|
||||
// Call the underlying parquetReader.next - which may replace the passed in ArrayWritable
|
||||
// with a new block of values
|
||||
boolean result = this.parquetReader.next(aVoid, arrayWritable);
|
||||
if (!result) {
|
||||
// if the result is false, then there are no more records
|
||||
return false;
|
||||
} else {
|
||||
// TODO(VC): Right now, we assume all records in log, have a matching base record. (which
|
||||
// would be true until we have a way to index logs too)
|
||||
// return from delta records map if we have some match.
|
||||
String key = arrayWritable.get()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS]
|
||||
.toString();
|
||||
if (deltaRecordMap.containsKey(key)) {
|
||||
// TODO(NA): Invoke preCombine here by converting arrayWritable to Avro. This is required since the
|
||||
// deltaRecord may not be a full record and needs values of columns from the parquet
|
||||
Option<GenericRecord> rec;
|
||||
if (usesCustomPayload) {
|
||||
rec = deltaRecordMap.get(key).getData().getInsertValue(getWriterSchema());
|
||||
} else {
|
||||
rec = deltaRecordMap.get(key).getData().getInsertValue(getReaderSchema());
|
||||
}
|
||||
if (!rec.isPresent()) {
|
||||
// If the record is not present, this is a delete record using an empty payload so skip this base record
|
||||
// and move to the next record
|
||||
return next(aVoid, arrayWritable);
|
||||
}
|
||||
GenericRecord recordToReturn = rec.get();
|
||||
if (usesCustomPayload) {
|
||||
// If using a custom payload, return only the projection fields
|
||||
recordToReturn = HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(rec.get(), getReaderSchema());
|
||||
}
|
||||
// we assume, a later safe record in the log, is newer than what we have in the map &
|
||||
// replace it.
|
||||
ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(recordToReturn, getWriterSchema());
|
||||
Writable[] replaceValue = aWritable.get();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("key %s, base values: %s, log values: %s", key,
|
||||
arrayWritableToString(arrayWritable), arrayWritableToString(aWritable)));
|
||||
}
|
||||
Writable[] originalValue = arrayWritable.get();
|
||||
try {
|
||||
System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length);
|
||||
arrayWritable.set(originalValue);
|
||||
} catch (RuntimeException re) {
|
||||
LOG.error("Got exception when doing array copy", re);
|
||||
LOG.error("Base record :" + arrayWritableToString(arrayWritable));
|
||||
LOG.error("Log record :" + arrayWritableToString(aWritable));
|
||||
throw re;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public NullWritable createKey() {
|
||||
return parquetReader.createKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayWritable createValue() {
|
||||
return parquetReader.createValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
return parquetReader.getPos();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
parquetReader.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException {
|
||||
return parquetReader.getProgress();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,143 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
|
||||
import org.apache.hudi.hadoop.RecordReaderValueIterator;
|
||||
import org.apache.hudi.hadoop.SafeParquetRecordReaderWrapper;
|
||||
|
||||
class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader implements
|
||||
RecordReader<NullWritable, ArrayWritable> {
|
||||
|
||||
// Log Record unmerged scanner
|
||||
private final HoodieUnMergedLogRecordScanner logRecordScanner;
|
||||
|
||||
// Parquet record reader
|
||||
private final RecordReader<NullWritable, ArrayWritable> parquetReader;
|
||||
|
||||
// Parquet record iterator wrapper for the above reader
|
||||
private final RecordReaderValueIterator<NullWritable, ArrayWritable> parquetRecordsIterator;
|
||||
|
||||
// Executor that runs the above producers in parallel
|
||||
private final BoundedInMemoryExecutor<ArrayWritable, ArrayWritable, ?> executor;
|
||||
|
||||
// Iterator for the buffer consumer
|
||||
private final Iterator<ArrayWritable> iterator;
|
||||
|
||||
/**
|
||||
* Construct a Unmerged record reader that parallely consumes both parquet and log records and buffers for upstream
|
||||
* clients to consume
|
||||
*
|
||||
* @param split File split
|
||||
* @param job Job Configuration
|
||||
* @param realReader Parquet Reader
|
||||
*/
|
||||
public RealtimeUnmergedRecordReader(HoodieRealtimeFileSplit split, JobConf job,
|
||||
RecordReader<NullWritable, ArrayWritable> realReader) {
|
||||
super(split, job);
|
||||
this.parquetReader = new SafeParquetRecordReaderWrapper(realReader);
|
||||
// Iterator for consuming records from parquet file
|
||||
this.parquetRecordsIterator = new RecordReaderValueIterator<>(this.parquetReader);
|
||||
this.executor = new BoundedInMemoryExecutor<>(getMaxCompactionMemoryInBytes(), getParallelProducers(),
|
||||
Option.empty(), x -> x, new DefaultSizeEstimator<>());
|
||||
// Consumer of this record reader
|
||||
this.iterator = this.executor.getQueue().iterator();
|
||||
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(
|
||||
FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(),
|
||||
split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(), Boolean.valueOf(jobConf
|
||||
.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
|
||||
record -> {
|
||||
// convert Hoodie log record to Hadoop AvroWritable and buffer
|
||||
GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema()).get();
|
||||
ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, getWriterSchema());
|
||||
this.executor.getQueue().insertRecord(aWritable);
|
||||
});
|
||||
// Start reading and buffering
|
||||
this.executor.startProducers();
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup log and parquet reading in parallel. Both write to central buffer.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
private List<BoundedInMemoryQueueProducer<ArrayWritable>> getParallelProducers() {
|
||||
List<BoundedInMemoryQueueProducer<ArrayWritable>> producers = new ArrayList<>();
|
||||
producers.add(new FunctionBasedQueueProducer<>(buffer -> {
|
||||
logRecordScanner.scan();
|
||||
return null;
|
||||
}));
|
||||
producers.add(new IteratorBasedQueueProducer<>(parquetRecordsIterator));
|
||||
return producers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean next(NullWritable key, ArrayWritable value) throws IOException {
|
||||
if (!iterator.hasNext()) {
|
||||
return false;
|
||||
}
|
||||
// Copy from buffer iterator and set to passed writable
|
||||
value.set(iterator.next().get());
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NullWritable createKey() {
|
||||
return parquetReader.createKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayWritable createValue() {
|
||||
return parquetReader.createValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
//TODO: vb - No logical way to represent parallel stream pos in a single long.
|
||||
// Should we just return invalid (-1). Where is it used ?
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
this.parquetRecordsIterator.close();
|
||||
this.executor.shutdownNow();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException {
|
||||
return Math.min(parquetReader.getProgress(), logRecordScanner.getProgress());
|
||||
}
|
||||
}
|
||||
614
hudi-hadoop-mr/src/main/resources/META-INF/LICENSE.txt
Normal file
614
hudi-hadoop-mr/src/main/resources/META-INF/LICENSE.txt
Normal file
@@ -0,0 +1,614 @@
|
||||
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "[]"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright [yyyy] [name of copyright owner]
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
||||
----
|
||||
This project bundles portions of the 'JQuery' project under the terms of the MIT license.
|
||||
|
||||
Copyright 2012 jQuery Foundation and other contributors
|
||||
http://jquery.com/
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining
|
||||
a copy of this software and associated documentation files (the
|
||||
"Software"), to deal in the Software without restriction, including
|
||||
without limitation the rights to use, copy, modify, merge, publish,
|
||||
distribute, sublicense, and/or sell copies of the Software, and to
|
||||
permit persons to whom the Software is furnished to do so, subject to
|
||||
the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be
|
||||
included in all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
|
||||
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
|
||||
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
|
||||
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
|
||||
LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
|
||||
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
|
||||
WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
|
||||
|
||||
----
|
||||
This project bundles a derivative of portions of the 'Asciidoctor' project
|
||||
under the terms of the MIT license.
|
||||
|
||||
The MIT License
|
||||
Copyright (C) 2012-2015 Dan Allen, Ryan Waldron and the Asciidoctor Project
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining a copy
|
||||
of this software and associated documentation files (the "Software"), to deal
|
||||
in the Software without restriction, including without limitation the rights
|
||||
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
|
||||
copies of the Software, and to permit persons to whom the Software is
|
||||
furnished to do so, subject to the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be included in
|
||||
all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
|
||||
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
|
||||
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
|
||||
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
|
||||
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
|
||||
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
|
||||
THE SOFTWARE.
|
||||
|
||||
----
|
||||
This project incorporates portions of the 'Protocol Buffers' project avaialble
|
||||
under a '3-clause BSD' license.
|
||||
|
||||
Copyright 2008, Google Inc.
|
||||
All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without
|
||||
modification, are permitted provided that the following conditions are
|
||||
met:
|
||||
|
||||
* Redistributions of source code must retain the above copyright
|
||||
notice, this list of conditions and the following disclaimer.
|
||||
* Redistributions in binary form must reproduce the above
|
||||
copyright notice, this list of conditions and the following disclaimer
|
||||
in the documentation and/or other materials provided with the
|
||||
distribution.
|
||||
* Neither the name of Google Inc. nor the names of its
|
||||
contributors may be used to endorse or promote products derived from
|
||||
this software without specific prior written permission.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
||||
Code generated by the Protocol Buffer compiler is owned by the owner
|
||||
of the input file used when generating it. This code is not
|
||||
standalone and requires a support library to be linked with it. This
|
||||
support library is itself covered by the above license.
|
||||
|
||||
----
|
||||
This project bundles a derivative image for our Orca Logo. This image is
|
||||
available under the Creative Commons By Attribution 3.0 License.
|
||||
|
||||
Creative Commons Legal Code
|
||||
|
||||
Attribution 3.0 Unported
|
||||
|
||||
CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE
|
||||
LEGAL SERVICES. DISTRIBUTION OF THIS LICENSE DOES NOT CREATE AN
|
||||
ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS
|
||||
INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES
|
||||
REGARDING THE INFORMATION PROVIDED, AND DISCLAIMS LIABILITY FOR
|
||||
DAMAGES RESULTING FROM ITS USE.
|
||||
|
||||
License
|
||||
|
||||
THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE
|
||||
COMMONS PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY
|
||||
COPYRIGHT AND/OR OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS
|
||||
AUTHORIZED UNDER THIS LICENSE OR COPYRIGHT LAW IS PROHIBITED.
|
||||
|
||||
BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE
|
||||
TO BE BOUND BY THE TERMS OF THIS LICENSE. TO THE EXTENT THIS LICENSE MAY
|
||||
BE CONSIDERED TO BE A CONTRACT, THE LICENSOR GRANTS YOU THE RIGHTS
|
||||
CONTAINED HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND
|
||||
CONDITIONS.
|
||||
|
||||
1. Definitions
|
||||
|
||||
a. "Adaptation" means a work based upon the Work, or upon the Work and
|
||||
other pre-existing works, such as a translation, adaptation,
|
||||
derivative work, arrangement of music or other alterations of a
|
||||
literary or artistic work, or phonogram or performance and includes
|
||||
cinematographic adaptations or any other form in which the Work may be
|
||||
recast, transformed, or adapted including in any form recognizably
|
||||
derived from the original, except that a work that constitutes a
|
||||
Collection will not be considered an Adaptation for the purpose of
|
||||
this License. For the avoidance of doubt, where the Work is a musical
|
||||
work, performance or phonogram, the synchronization of the Work in
|
||||
timed-relation with a moving image ("synching") will be considered an
|
||||
Adaptation for the purpose of this License.
|
||||
b. "Collection" means a collection of literary or artistic works, such as
|
||||
encyclopedias and anthologies, or performances, phonograms or
|
||||
broadcasts, or other works or subject matter other than works listed
|
||||
in Section 1(f) below, which, by reason of the selection and
|
||||
arrangement of their contents, constitute intellectual creations, in
|
||||
which the Work is included in its entirety in unmodified form along
|
||||
with one or more other contributions, each constituting separate and
|
||||
independent works in themselves, which together are assembled into a
|
||||
collective whole. A work that constitutes a Collection will not be
|
||||
considered an Adaptation (as defined above) for the purposes of this
|
||||
License.
|
||||
c. "Distribute" means to make available to the public the original and
|
||||
copies of the Work or Adaptation, as appropriate, through sale or
|
||||
other transfer of ownership.
|
||||
d. "Licensor" means the individual, individuals, entity or entities that
|
||||
offer(s) the Work under the terms of this License.
|
||||
e. "Original Author" means, in the case of a literary or artistic work,
|
||||
the individual, individuals, entity or entities who created the Work
|
||||
or if no individual or entity can be identified, the publisher; and in
|
||||
addition (i) in the case of a performance the actors, singers,
|
||||
musicians, dancers, and other persons who act, sing, deliver, declaim,
|
||||
play in, interpret or otherwise perform literary or artistic works or
|
||||
expressions of folklore; (ii) in the case of a phonogram the producer
|
||||
being the person or legal entity who first fixes the sounds of a
|
||||
performance or other sounds; and, (iii) in the case of broadcasts, the
|
||||
organization that transmits the broadcast.
|
||||
f. "Work" means the literary and/or artistic work offered under the terms
|
||||
of this License including without limitation any production in the
|
||||
literary, scientific and artistic domain, whatever may be the mode or
|
||||
form of its expression including digital form, such as a book,
|
||||
pamphlet and other writing; a lecture, address, sermon or other work
|
||||
of the same nature; a dramatic or dramatico-musical work; a
|
||||
choreographic work or entertainment in dumb show; a musical
|
||||
composition with or without words; a cinematographic work to which are
|
||||
assimilated works expressed by a process analogous to cinematography;
|
||||
a work of drawing, painting, architecture, sculpture, engraving or
|
||||
lithography; a photographic work to which are assimilated works
|
||||
expressed by a process analogous to photography; a work of applied
|
||||
art; an illustration, map, plan, sketch or three-dimensional work
|
||||
relative to geography, topography, architecture or science; a
|
||||
performance; a broadcast; a phonogram; a compilation of data to the
|
||||
extent it is protected as a copyrightable work; or a work performed by
|
||||
a variety or circus performer to the extent it is not otherwise
|
||||
considered a literary or artistic work.
|
||||
g. "You" means an individual or entity exercising rights under this
|
||||
License who has not previously violated the terms of this License with
|
||||
respect to the Work, or who has received express permission from the
|
||||
Licensor to exercise rights under this License despite a previous
|
||||
violation.
|
||||
h. "Publicly Perform" means to perform public recitations of the Work and
|
||||
to communicate to the public those public recitations, by any means or
|
||||
process, including by wire or wireless means or public digital
|
||||
performances; to make available to the public Works in such a way that
|
||||
members of the public may access these Works from a place and at a
|
||||
place individually chosen by them; to perform the Work to the public
|
||||
by any means or process and the communication to the public of the
|
||||
performances of the Work, including by public digital performance; to
|
||||
broadcast and rebroadcast the Work by any means including signs,
|
||||
sounds or images.
|
||||
i. "Reproduce" means to make copies of the Work by any means including
|
||||
without limitation by sound or visual recordings and the right of
|
||||
fixation and reproducing fixations of the Work, including storage of a
|
||||
protected performance or phonogram in digital form or other electronic
|
||||
medium.
|
||||
|
||||
2. Fair Dealing Rights. Nothing in this License is intended to reduce,
|
||||
limit, or restrict any uses free from copyright or rights arising from
|
||||
limitations or exceptions that are provided for in connection with the
|
||||
copyright protection under copyright law or other applicable laws.
|
||||
|
||||
3. License Grant. Subject to the terms and conditions of this License,
|
||||
Licensor hereby grants You a worldwide, royalty-free, non-exclusive,
|
||||
perpetual (for the duration of the applicable copyright) license to
|
||||
exercise the rights in the Work as stated below:
|
||||
|
||||
a. to Reproduce the Work, to incorporate the Work into one or more
|
||||
Collections, and to Reproduce the Work as incorporated in the
|
||||
Collections;
|
||||
b. to create and Reproduce Adaptations provided that any such Adaptation,
|
||||
including any translation in any medium, takes reasonable steps to
|
||||
clearly label, demarcate or otherwise identify that changes were made
|
||||
to the original Work. For example, a translation could be marked "The
|
||||
original work was translated from English to Spanish," or a
|
||||
modification could indicate "The original work has been modified.";
|
||||
c. to Distribute and Publicly Perform the Work including as incorporated
|
||||
in Collections; and,
|
||||
d. to Distribute and Publicly Perform Adaptations.
|
||||
e. For the avoidance of doubt:
|
||||
|
||||
i. Non-waivable Compulsory License Schemes. In those jurisdictions in
|
||||
which the right to collect royalties through any statutory or
|
||||
compulsory licensing scheme cannot be waived, the Licensor
|
||||
reserves the exclusive right to collect such royalties for any
|
||||
exercise by You of the rights granted under this License;
|
||||
ii. Waivable Compulsory License Schemes. In those jurisdictions in
|
||||
which the right to collect royalties through any statutory or
|
||||
compulsory licensing scheme can be waived, the Licensor waives the
|
||||
exclusive right to collect such royalties for any exercise by You
|
||||
of the rights granted under this License; and,
|
||||
iii. Voluntary License Schemes. The Licensor waives the right to
|
||||
collect royalties, whether individually or, in the event that the
|
||||
Licensor is a member of a collecting society that administers
|
||||
voluntary licensing schemes, via that society, from any exercise
|
||||
by You of the rights granted under this License.
|
||||
|
||||
The above rights may be exercised in all media and formats whether now
|
||||
known or hereafter devised. The above rights include the right to make
|
||||
such modifications as are technically necessary to exercise the rights in
|
||||
other media and formats. Subject to Section 8(f), all rights not expressly
|
||||
granted by Licensor are hereby reserved.
|
||||
|
||||
4. Restrictions. The license granted in Section 3 above is expressly made
|
||||
subject to and limited by the following restrictions:
|
||||
|
||||
a. You may Distribute or Publicly Perform the Work only under the terms
|
||||
of this License. You must include a copy of, or the Uniform Resource
|
||||
Identifier (URI) for, this License with every copy of the Work You
|
||||
Distribute or Publicly Perform. You may not offer or impose any terms
|
||||
on the Work that restrict the terms of this License or the ability of
|
||||
the recipient of the Work to exercise the rights granted to that
|
||||
recipient under the terms of the License. You may not sublicense the
|
||||
Work. You must keep intact all notices that refer to this License and
|
||||
to the disclaimer of warranties with every copy of the Work You
|
||||
Distribute or Publicly Perform. When You Distribute or Publicly
|
||||
Perform the Work, You may not impose any effective technological
|
||||
measures on the Work that restrict the ability of a recipient of the
|
||||
Work from You to exercise the rights granted to that recipient under
|
||||
the terms of the License. This Section 4(a) applies to the Work as
|
||||
incorporated in a Collection, but this does not require the Collection
|
||||
apart from the Work itself to be made subject to the terms of this
|
||||
License. If You create a Collection, upon notice from any Licensor You
|
||||
must, to the extent practicable, remove from the Collection any credit
|
||||
as required by Section 4(b), as requested. If You create an
|
||||
Adaptation, upon notice from any Licensor You must, to the extent
|
||||
practicable, remove from the Adaptation any credit as required by
|
||||
Section 4(b), as requested.
|
||||
b. If You Distribute, or Publicly Perform the Work or any Adaptations or
|
||||
Collections, You must, unless a request has been made pursuant to
|
||||
Section 4(a), keep intact all copyright notices for the Work and
|
||||
provide, reasonable to the medium or means You are utilizing: (i) the
|
||||
name of the Original Author (or pseudonym, if applicable) if supplied,
|
||||
and/or if the Original Author and/or Licensor designate another party
|
||||
or parties (e.g., a sponsor institute, publishing entity, journal) for
|
||||
attribution ("Attribution Parties") in Licensor's copyright notice,
|
||||
terms of service or by other reasonable means, the name of such party
|
||||
or parties; (ii) the title of the Work if supplied; (iii) to the
|
||||
extent reasonably practicable, the URI, if any, that Licensor
|
||||
specifies to be associated with the Work, unless such URI does not
|
||||
refer to the copyright notice or licensing information for the Work;
|
||||
and (iv) , consistent with Section 3(b), in the case of an Adaptation,
|
||||
a credit identifying the use of the Work in the Adaptation (e.g.,
|
||||
"French translation of the Work by Original Author," or "Screenplay
|
||||
based on original Work by Original Author"). The credit required by
|
||||
this Section 4 (b) may be implemented in any reasonable manner;
|
||||
provided, however, that in the case of a Adaptation or Collection, at
|
||||
a minimum such credit will appear, if a credit for all contributing
|
||||
authors of the Adaptation or Collection appears, then as part of these
|
||||
credits and in a manner at least as prominent as the credits for the
|
||||
other contributing authors. For the avoidance of doubt, You may only
|
||||
use the credit required by this Section for the purpose of attribution
|
||||
in the manner set out above and, by exercising Your rights under this
|
||||
License, You may not implicitly or explicitly assert or imply any
|
||||
connection with, sponsorship or endorsement by the Original Author,
|
||||
Licensor and/or Attribution Parties, as appropriate, of You or Your
|
||||
use of the Work, without the separate, express prior written
|
||||
permission of the Original Author, Licensor and/or Attribution
|
||||
Parties.
|
||||
c. Except as otherwise agreed in writing by the Licensor or as may be
|
||||
otherwise permitted by applicable law, if You Reproduce, Distribute or
|
||||
Publicly Perform the Work either by itself or as part of any
|
||||
Adaptations or Collections, You must not distort, mutilate, modify or
|
||||
take other derogatory action in relation to the Work which would be
|
||||
prejudicial to the Original Author's honor or reputation. Licensor
|
||||
agrees that in those jurisdictions (e.g. Japan), in which any exercise
|
||||
of the right granted in Section 3(b) of this License (the right to
|
||||
make Adaptations) would be deemed to be a distortion, mutilation,
|
||||
modification or other derogatory action prejudicial to the Original
|
||||
Author's honor and reputation, the Licensor will waive or not assert,
|
||||
as appropriate, this Section, to the fullest extent permitted by the
|
||||
applicable national law, to enable You to reasonably exercise Your
|
||||
right under Section 3(b) of this License (right to make Adaptations)
|
||||
but not otherwise.
|
||||
|
||||
5. Representations, Warranties and Disclaimer
|
||||
|
||||
UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR
|
||||
OFFERS THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY
|
||||
KIND CONCERNING THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE,
|
||||
INCLUDING, WITHOUT LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY,
|
||||
FITNESS FOR A PARTICULAR PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF
|
||||
LATENT OR OTHER DEFECTS, ACCURACY, OR THE PRESENCE OF ABSENCE OF ERRORS,
|
||||
WHETHER OR NOT DISCOVERABLE. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION
|
||||
OF IMPLIED WARRANTIES, SO SUCH EXCLUSION MAY NOT APPLY TO YOU.
|
||||
|
||||
6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE
|
||||
LAW, IN NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR
|
||||
ANY SPECIAL, INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES
|
||||
ARISING OUT OF THIS LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS
|
||||
BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
|
||||
|
||||
7. Termination
|
||||
|
||||
a. This License and the rights granted hereunder will terminate
|
||||
automatically upon any breach by You of the terms of this License.
|
||||
Individuals or entities who have received Adaptations or Collections
|
||||
from You under this License, however, will not have their licenses
|
||||
terminated provided such individuals or entities remain in full
|
||||
compliance with those licenses. Sections 1, 2, 5, 6, 7, and 8 will
|
||||
survive any termination of this License.
|
||||
b. Subject to the above terms and conditions, the license granted here is
|
||||
perpetual (for the duration of the applicable copyright in the Work).
|
||||
Notwithstanding the above, Licensor reserves the right to release the
|
||||
Work under different license terms or to stop distributing the Work at
|
||||
any time; provided, however that any such election will not serve to
|
||||
withdraw this License (or any other license that has been, or is
|
||||
required to be, granted under the terms of this License), and this
|
||||
License will continue in full force and effect unless terminated as
|
||||
stated above.
|
||||
|
||||
8. Miscellaneous
|
||||
|
||||
a. Each time You Distribute or Publicly Perform the Work or a Collection,
|
||||
the Licensor offers to the recipient a license to the Work on the same
|
||||
terms and conditions as the license granted to You under this License.
|
||||
b. Each time You Distribute or Publicly Perform an Adaptation, Licensor
|
||||
offers to the recipient a license to the original Work on the same
|
||||
terms and conditions as the license granted to You under this License.
|
||||
c. If any provision of this License is invalid or unenforceable under
|
||||
applicable law, it shall not affect the validity or enforceability of
|
||||
the remainder of the terms of this License, and without further action
|
||||
by the parties to this agreement, such provision shall be reformed to
|
||||
the minimum extent necessary to make such provision valid and
|
||||
enforceable.
|
||||
d. No term or provision of this License shall be deemed waived and no
|
||||
breach consented to unless such waiver or consent shall be in writing
|
||||
and signed by the party to be charged with such waiver or consent.
|
||||
e. This License constitutes the entire agreement between the parties with
|
||||
respect to the Work licensed here. There are no understandings,
|
||||
agreements or representations with respect to the Work not specified
|
||||
here. Licensor shall not be bound by any additional provisions that
|
||||
may appear in any communication from You. This License may not be
|
||||
modified without the mutual written agreement of the Licensor and You.
|
||||
f. The rights granted under, and the subject matter referenced, in this
|
||||
License were drafted utilizing the terminology of the Berne Convention
|
||||
for the Protection of Literary and Artistic Works (as amended on
|
||||
September 28, 1979), the Rome Convention of 1961, the WIPO Copyright
|
||||
Treaty of 1996, the WIPO Performances and Phonograms Treaty of 1996
|
||||
and the Universal Copyright Convention (as revised on July 24, 1971).
|
||||
These rights and subject matter take effect in the relevant
|
||||
jurisdiction in which the License terms are sought to be enforced
|
||||
according to the corresponding provisions of the implementation of
|
||||
those treaty provisions in the applicable national law. If the
|
||||
standard suite of rights granted under applicable copyright law
|
||||
includes additional rights not granted under this License, such
|
||||
additional rights are deemed to be included in the License; this
|
||||
License is not intended to restrict the license of any rights under
|
||||
applicable law.
|
||||
|
||||
|
||||
Creative Commons Notice
|
||||
|
||||
Creative Commons is not a party to this License, and makes no warranty
|
||||
whatsoever in connection with the Work. Creative Commons will not be
|
||||
liable to You or any party on any legal theory for any damages
|
||||
whatsoever, including without limitation any general, special,
|
||||
incidental or consequential damages arising in connection to this
|
||||
license. Notwithstanding the foregoing two (2) sentences, if Creative
|
||||
Commons has expressly identified itself as the Licensor hereunder, it
|
||||
shall have all rights and obligations of Licensor.
|
||||
|
||||
Except for the limited purpose of indicating to the public that the
|
||||
Work is licensed under the CCPL, Creative Commons does not authorize
|
||||
the use by either party of the trademark "Creative Commons" or any
|
||||
related trademark or logo of Creative Commons without the prior
|
||||
written consent of Creative Commons. Any permitted use will be in
|
||||
compliance with Creative Commons' then-current trademark usage
|
||||
guidelines, as may be published on its website or otherwise made
|
||||
available upon request from time to time. For the avoidance of doubt,
|
||||
this trademark restriction does not form part of this License.
|
||||
|
||||
Creative Commons may be contacted at https://creativecommons.org/.
|
||||
235
hudi-hadoop-mr/src/main/resources/META-INF/NOTICE.txt
Normal file
235
hudi-hadoop-mr/src/main/resources/META-INF/NOTICE.txt
Normal file
@@ -0,0 +1,235 @@
|
||||
Apache HUDI
|
||||
Copyright 2019 The Apache Software Foundation
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.
|
||||
|
||||
This project includes:
|
||||
aircompressor under Apache License 2.0
|
||||
An open source Java toolkit for Amazon S3 under Apache License, Version 2.0
|
||||
Annotation 1.0 under The Apache Software License, Version 2.0
|
||||
ant under The Apache Software License, Version 2.0
|
||||
ANTLR 3 Runtime under BSD licence
|
||||
ANTLR ST4 4.0.4 under BSD licence
|
||||
AOP alliance under Public Domain
|
||||
Apache Ant Core under The Apache Software License, Version 2.0
|
||||
Apache Ant Launcher under The Apache Software License, Version 2.0
|
||||
Apache Avro under The Apache Software License, Version 2.0
|
||||
Apache Avro IPC under The Apache Software License, Version 2.0
|
||||
Apache Avro Mapred API under The Apache Software License, Version 2.0
|
||||
Apache Calcite Avatica under Apache License, Version 2.0
|
||||
Apache Calcite Avatica Metrics under Apache License, Version 2.0
|
||||
Apache Commons Collections under Apache License, Version 2.0
|
||||
Apache Commons IO under Apache License, Version 2.0
|
||||
Apache Commons Logging under The Apache Software License, Version 2.0
|
||||
Apache Curator under The Apache Software License, Version 2.0
|
||||
Apache Derby Database Engine and Embedded JDBC Driver under Apache 2
|
||||
Apache Directory API ASN.1 API under The Apache Software License, Version 2.0
|
||||
Apache Directory LDAP API Utilities under The Apache Software License, Version 2.0
|
||||
Apache Groovy under The Apache Software License, Version 2.0
|
||||
Apache Hadoop Annotations under Apache License, Version 2.0
|
||||
Apache Hadoop Auth under Apache License, Version 2.0
|
||||
Apache Hadoop Client under Apache License, Version 2.0
|
||||
Apache Hadoop Common under Apache License, Version 2.0
|
||||
Apache Hadoop HDFS under Apache License, Version 2.0
|
||||
Apache HBase - Annotations under Apache License, Version 2.0
|
||||
Apache HBase - Client under Apache License, Version 2.0
|
||||
Apache HBase - Protocol under Apache License, Version 2.0
|
||||
Apache HttpClient under Apache License, Version 2.0
|
||||
Apache HttpCore under Apache License, Version 2.0
|
||||
Apache Ivy under The Apache Software License, Version 2.0
|
||||
Apache Log4j under The Apache Software License, Version 2.0
|
||||
Apache Log4j 1.x Compatibility API under The Apache Software License, Version 2.0
|
||||
Apache Log4j API under The Apache Software License, Version 2.0
|
||||
Apache Log4j Core under The Apache Software License, Version 2.0
|
||||
Apache Log4j SLF4J Binding under The Apache Software License, Version 2.0
|
||||
Apache Log4j Web under The Apache Software License, Version 2.0
|
||||
Apache Parquet Avro under The Apache Software License, Version 2.0
|
||||
Apache Parquet Avro (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Column under The Apache Software License, Version 2.0
|
||||
Apache Parquet Column (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Common under The Apache Software License, Version 2.0
|
||||
Apache Parquet Common (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Encodings under The Apache Software License, Version 2.0
|
||||
Apache Parquet Encodings (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Format (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Generator (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop Bundle under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop Bundle (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Jackson under The Apache Software License, Version 2.0
|
||||
Apache Parquet Jackson (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Thrift under The Apache Software License, Version 2.0
|
||||
Apache Twill API under The Apache Software License, Version 2.0
|
||||
Apache Twill common library under The Apache Software License, Version 2.0
|
||||
Apache Twill core library under The Apache Software License, Version 2.0
|
||||
Apache Twill discovery service API under The Apache Software License, Version 2.0
|
||||
Apache Twill discovery service implementations under The Apache Software License, Version 2.0
|
||||
Apache Twill ZooKeeper client library under The Apache Software License, Version 2.0
|
||||
Apache Velocity under The Apache Software License, Version 2.0
|
||||
ApacheDS I18n under The Apache Software License, Version 2.0
|
||||
ApacheDS Protocol Kerberos Codec under The Apache Software License, Version 2.0
|
||||
ASM Commons under 3-Clause BSD License
|
||||
ASM Core under 3-Clause BSD License
|
||||
ASM Tree under 3-Clause BSD License
|
||||
BoneCP :: Core Library under Apache v2
|
||||
Calcite Core under Apache License, Version 2.0
|
||||
Calcite Druid under Apache License, Version 2.0
|
||||
Calcite Linq4j under Apache License, Version 2.0
|
||||
com.twitter.common:objectsize under Apache License, Version 2.0
|
||||
Commons BeanUtils Core under The Apache Software License, Version 2.0
|
||||
Commons CLI under The Apache Software License, Version 2.0
|
||||
Commons Codec under The Apache Software License, Version 2.0
|
||||
Commons Compiler under New BSD License
|
||||
Commons Compress under The Apache Software License, Version 2.0
|
||||
Commons Configuration under The Apache Software License, Version 2.0
|
||||
Commons Daemon under The Apache Software License, Version 2.0
|
||||
Commons DBCP under The Apache Software License, Version 2.0
|
||||
Commons Lang under The Apache Software License, Version 2.0
|
||||
Commons Math under The Apache Software License, Version 2.0
|
||||
Commons Net under The Apache Software License, Version 2.0
|
||||
Commons Pool under The Apache Software License, Version 2.0
|
||||
commons-beanutils under Apache License
|
||||
Curator Client under The Apache Software License, Version 2.0
|
||||
Curator Framework under The Apache Software License, Version 2.0
|
||||
Curator Recipes under The Apache Software License, Version 2.0
|
||||
Data Mapper for Jackson under The Apache Software License, Version 2.0
|
||||
DataNucleus Core under The Apache Software License, Version 2.0
|
||||
DataNucleus JDO API plugin under The Apache Software License, Version 2.0
|
||||
DataNucleus RDBMS plugin under The Apache Software License, Version 2.0
|
||||
Digester under The Apache Software License, Version 2.0
|
||||
Disruptor Framework under The Apache Software License, Version 2.0
|
||||
eigenbase-properties under Apache License, Version 2.0
|
||||
EL under The Apache Software License, Version 2.0
|
||||
fastutil under Apache License, Version 2.0
|
||||
Findbugs Annotations under Apache License under Apache License, Version 2.0
|
||||
FindBugs-jsr305 under The Apache Software License, Version 2.0
|
||||
Fluent API for Apache HttpClient under Apache License, Version 2.0
|
||||
Glassfish Jasper under CDDL 1.0
|
||||
Glassfish Jasper API under Apache License Version 2.0
|
||||
Google Guice - Core Library under The Apache Software License, Version 2.0
|
||||
Google Guice - Extensions - AssistedInject under The Apache Software License, Version 2.0
|
||||
Google Guice - Extensions - Servlet under The Apache Software License, Version 2.0
|
||||
Gson under The Apache Software License, Version 2.0
|
||||
Guava: Google Core Libraries for Java under The Apache Software License, Version 2.0
|
||||
Hadoop Metrics2 Reporter for Dropwizard Metrics under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-app under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-common under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-core under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-jobclient under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-shuffle under Apache License, Version 2.0
|
||||
hadoop-yarn-api under Apache License, Version 2.0
|
||||
hadoop-yarn-client under Apache License, Version 2.0
|
||||
hadoop-yarn-common under Apache License, Version 2.0
|
||||
hadoop-yarn-registry under Apache License, Version 2.0
|
||||
hadoop-yarn-server-applicationhistoryservice under Apache License, Version 2.0
|
||||
hadoop-yarn-server-common under Apache License, Version 2.0
|
||||
hadoop-yarn-server-resourcemanager under Apache License, Version 2.0
|
||||
hadoop-yarn-server-web-proxy under Apache License, Version 2.0
|
||||
Hamcrest Core under New BSD License
|
||||
HBase - Common under The Apache Software License, Version 2.0
|
||||
HBase - Hadoop Compatibility under The Apache Software License, Version 2.0
|
||||
HBase - Hadoop Two Compatibility under The Apache Software License, Version 2.0
|
||||
HBase - Prefix Tree under The Apache Software License, Version 2.0
|
||||
HBase - Procedure under The Apache Software License, Version 2.0
|
||||
HBase - Server under The Apache Software License, Version 2.0
|
||||
HikariCP under The Apache Software License, Version 2.0
|
||||
Hive Common under The Apache Software License, Version 2.0
|
||||
Hive JDBC under The Apache Software License, Version 2.0
|
||||
Hive Llap Client under The Apache Software License, Version 2.0
|
||||
Hive Llap Common under The Apache Software License, Version 2.0
|
||||
Hive Llap Server under The Apache Software License, Version 2.0
|
||||
Hive Llap Tez under The Apache Software License, Version 2.0
|
||||
Hive Metastore under The Apache Software License, Version 2.0
|
||||
Hive Query Language under The Apache Software License, Version 2.0
|
||||
Hive Serde under The Apache Software License, Version 2.0
|
||||
Hive Service under The Apache Software License, Version 2.0
|
||||
Hive Service RPC under The Apache Software License, Version 2.0
|
||||
Hive Shims under The Apache Software License, Version 2.0
|
||||
Hive Shims 0.23 under The Apache Software License, Version 2.0
|
||||
Hive Shims Common under The Apache Software License, Version 2.0
|
||||
Hive Shims Scheduler under The Apache Software License, Version 2.0
|
||||
Hive Storage API under Apache License, Version 2.0
|
||||
Hive Vector-Code-Gen Utilities under The Apache Software License, Version 2.0
|
||||
hoodie-common under Apache License, Version 2.0
|
||||
hoodie-hadoop-mr under Apache License, Version 2.0
|
||||
htrace-core under The Apache Software License, Version 2.0
|
||||
HttpClient under Apache License
|
||||
Jackson under The Apache Software License, Version 2.0
|
||||
Jackson Integration for Metrics under Apache License 2.0
|
||||
Jackson-annotations under The Apache Software License, Version 2.0
|
||||
Jackson-core under The Apache Software License, Version 2.0
|
||||
jackson-databind under The Apache Software License, Version 2.0
|
||||
jamon-runtime under Mozilla Public License Version 1.1
|
||||
Janino under New BSD License
|
||||
jasper-compiler under The Apache Software License, Version 2.0
|
||||
jasper-runtime under The Apache Software License, Version 2.0
|
||||
Java Authentication SPI for Containers under The Apache Software License, Version 2.0
|
||||
java-xmlbuilder under Apache License, Version 2.0
|
||||
JavaBeans Activation Framework (JAF) under Common Development and Distribution License (CDDL) v1.0
|
||||
javax.inject under The Apache Software License, Version 2.0
|
||||
Javolution under BSD License
|
||||
JAX-RS provider for JSON content type under The Apache Software License, Version 2.0 or GNU Lesser General Public License (LGPL), Version 2.1
|
||||
JAXB RI under CDDL 1.1 or GPL2 w/ CPE
|
||||
JCodings under MIT License
|
||||
jcommander under Apache 2.0
|
||||
JDO API under Apache 2
|
||||
jersey-client under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-core under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-guice under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-json under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-server under CDDL 1.1 or GPL2 w/ CPE
|
||||
Jettison under Apache License, Version 2.0
|
||||
Jetty :: Aggregate :: All core Jetty under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty Orbit :: Servlet API under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty Server under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty SSLEngine under Apache License Version 2
|
||||
Jetty Utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
JLine under The BSD License
|
||||
Joda-Time under Apache 2
|
||||
Joni under MIT License
|
||||
JPam under The Apache Software License, Version 2.0
|
||||
JSch under BSD
|
||||
jsp-api under CDDL
|
||||
JTA 1.1 under The Apache Software License, Version 2.0
|
||||
JUnit under Common Public License Version 1.0
|
||||
JVM Integration for Metrics under Apache License 2.0
|
||||
Kryo under New BSD License
|
||||
Kryo Shaded under 3-Clause BSD License
|
||||
leveldbjni-all under The BSD 3-Clause License
|
||||
Metrics Core under Apache License 2.0
|
||||
Metrics Core Library under Apache License 2.0
|
||||
MinLog under New BSD License
|
||||
Netty/All-in-One under Apache License, Version 2.0
|
||||
Objenesis under Apache 2
|
||||
Open JSON under The Apache Software License, Version 2.0
|
||||
opencsv under Apache 2
|
||||
ORC Core under Apache License, Version 2.0
|
||||
org.pentaho:pentaho-aggdesigner-algorithm under Apache License, Version 2.0
|
||||
ParaNamer Core under BSD
|
||||
Protocol Buffer Java API under New BSD license
|
||||
ReflectASM under New BSD License
|
||||
RocksDB JNI under Apache License 2.0 or GNU General Public License, version 2
|
||||
Servlet Specification 2.5 API under CDDL 1.0
|
||||
Servlet Specification API under Apache License Version 2.0
|
||||
servlet-api under CDDL
|
||||
SLF4J API Module under MIT License
|
||||
SLF4J LOG4J-12 Binding under MIT License
|
||||
Slider Core under Apache License, Version 2.0
|
||||
Snappy for Java under The Apache Software License, Version 2.0
|
||||
StAX API under The Apache Software License, Version 2.0
|
||||
Tephra API under The Apache Software License, Version 2.0
|
||||
Tephra Core under The Apache Software License, Version 2.0
|
||||
Tephra HBase 1.0 Compatibility under The Apache Software License, Version 2.0
|
||||
The Netty Project under Apache License, Version 2.0
|
||||
Xerces2 Java Parser under The Apache Software License, Version 2.0
|
||||
XML Commons External Components XML APIs under The Apache Software License, Version 2.0
|
||||
Xml Compatibility extensions for Jackson under The Apache Software License, Version 2.0 or GNU Lesser General Public License (LGPL), Version 2.1
|
||||
xmlenc Library under The BSD License
|
||||
XZ for Java under Public Domain
|
||||
zookeeper under Apache License, Version 2.0
|
||||
|
||||
Reference in New Issue
Block a user