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
77
hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java
Normal file
77
hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java
Normal file
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* 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.cli;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
|
||||
public class HoodieCLI {
|
||||
|
||||
public static Configuration conf;
|
||||
public static ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build();
|
||||
public static FileSystem fs;
|
||||
public static CLIState state = CLIState.INIT;
|
||||
public static String basePath;
|
||||
public static HoodieTableMetaClient tableMetadata;
|
||||
public static HoodieTableMetaClient syncTableMetadata;
|
||||
|
||||
|
||||
public enum CLIState {
|
||||
INIT, DATASET, SYNC
|
||||
}
|
||||
|
||||
public static void setConsistencyGuardConfig(ConsistencyGuardConfig config) {
|
||||
consistencyGuardConfig = config;
|
||||
}
|
||||
|
||||
private static void setTableMetaClient(HoodieTableMetaClient tableMetadata) {
|
||||
HoodieCLI.tableMetadata = tableMetadata;
|
||||
}
|
||||
|
||||
private static void setBasePath(String basePath) {
|
||||
HoodieCLI.basePath = basePath;
|
||||
}
|
||||
|
||||
public static boolean initConf() {
|
||||
if (HoodieCLI.conf == null) {
|
||||
HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration());
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static void initFS(boolean force) throws IOException {
|
||||
if (fs == null || force) {
|
||||
fs = (tableMetadata != null) ? tableMetadata.getFs() : FileSystem.get(conf);
|
||||
}
|
||||
}
|
||||
|
||||
public static void refreshTableMetadata() {
|
||||
setTableMetaClient(new HoodieTableMetaClient(HoodieCLI.conf, basePath, false, HoodieCLI.consistencyGuardConfig));
|
||||
}
|
||||
|
||||
public static void connectTo(String basePath) {
|
||||
setBasePath(basePath);
|
||||
refreshTableMetadata();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,39 @@
|
||||
/*
|
||||
* 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.cli;
|
||||
|
||||
import org.springframework.core.Ordered;
|
||||
import org.springframework.core.annotation.Order;
|
||||
import org.springframework.shell.plugin.support.DefaultHistoryFileNameProvider;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
@Order(Ordered.HIGHEST_PRECEDENCE)
|
||||
public class HoodieHistoryFileNameProvider extends DefaultHistoryFileNameProvider {
|
||||
|
||||
public String getHistoryFileName() {
|
||||
return "hoodie-cmd.log";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getProviderName() {
|
||||
return "Hoodie file name provider";
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,109 @@
|
||||
/*
|
||||
* 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.cli;
|
||||
|
||||
import com.jakewharton.fliptables.FlipTable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Helper class to render table for hoodie-cli
|
||||
*/
|
||||
public class HoodiePrintHelper {
|
||||
|
||||
/**
|
||||
* Print header and raw rows
|
||||
*
|
||||
* @param header Header
|
||||
* @param rows Raw Rows
|
||||
* @return output
|
||||
*/
|
||||
public static String print(String[] header, String[][] rows) {
|
||||
return printTextTable(header, rows);
|
||||
}
|
||||
|
||||
/**
|
||||
* Serialize Table to printable string
|
||||
*
|
||||
* @param rowHeader Row Header
|
||||
* @param fieldNameToConverterMap Field Specific Converters
|
||||
* @param sortByField Sorting field
|
||||
* @param isDescending Order
|
||||
* @param limit Limit
|
||||
* @param headerOnly Headers only
|
||||
* @param rows List of rows
|
||||
* @return Serialized form for printing
|
||||
*/
|
||||
public static String print(TableHeader rowHeader,
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap,
|
||||
String sortByField, boolean isDescending, Integer limit, boolean headerOnly,
|
||||
List<Comparable[]> rows) {
|
||||
|
||||
if (headerOnly) {
|
||||
return HoodiePrintHelper.print(rowHeader);
|
||||
}
|
||||
|
||||
Table table = new Table(rowHeader, fieldNameToConverterMap,
|
||||
Option.ofNullable(sortByField.isEmpty() ? null : sortByField),
|
||||
Option.ofNullable(isDescending),
|
||||
Option.ofNullable(limit <= 0 ? null : limit)).addAllRows(rows).flip();
|
||||
|
||||
return HoodiePrintHelper.print(table);
|
||||
}
|
||||
|
||||
/**
|
||||
* Render rows in Table
|
||||
*
|
||||
* @param buffer Table
|
||||
* @return output
|
||||
*/
|
||||
private static String print(Table buffer) {
|
||||
String[] header = new String[buffer.getFieldNames().size()];
|
||||
buffer.getFieldNames().toArray(header);
|
||||
|
||||
String[][] rows = buffer.getRenderRows().stream()
|
||||
.map(l -> l.stream().toArray(String[]::new))
|
||||
.toArray(String[][]::new);
|
||||
return printTextTable(header, rows);
|
||||
}
|
||||
|
||||
/**
|
||||
* Render only header of the table
|
||||
*
|
||||
* @param header Table Header
|
||||
* @return output
|
||||
*/
|
||||
private static String print(TableHeader header) {
|
||||
String[] head = new String[header.getFieldNames().size()];
|
||||
header.getFieldNames().toArray(head);
|
||||
return printTextTable(head, new String[][]{});
|
||||
}
|
||||
|
||||
/**
|
||||
* Print Text table
|
||||
*
|
||||
* @param headers Headers
|
||||
* @param data Table
|
||||
*/
|
||||
private static String printTextTable(String[] headers, String[][] data) {
|
||||
return FlipTable.of(headers, data);
|
||||
}
|
||||
}
|
||||
53
hudi-cli/src/main/java/org/apache/hudi/cli/HoodiePrompt.java
Normal file
53
hudi-cli/src/main/java/org/apache/hudi/cli/HoodiePrompt.java
Normal file
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* 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.cli;
|
||||
|
||||
import org.springframework.core.Ordered;
|
||||
import org.springframework.core.annotation.Order;
|
||||
import org.springframework.shell.plugin.support.DefaultPromptProvider;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
@Order(Ordered.HIGHEST_PRECEDENCE)
|
||||
public class HoodiePrompt extends DefaultPromptProvider {
|
||||
|
||||
@Override
|
||||
public String getPrompt() {
|
||||
if (HoodieCLI.tableMetadata != null) {
|
||||
String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName();
|
||||
switch (HoodieCLI.state) {
|
||||
case INIT:
|
||||
return "hudi->";
|
||||
case DATASET:
|
||||
return "hudi:" + tableName + "->";
|
||||
case SYNC:
|
||||
return "hudi:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
|
||||
default:
|
||||
return "hudi:" + tableName + "->";
|
||||
}
|
||||
}
|
||||
return "hudi->";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getProviderName() {
|
||||
return "Hoodie provider";
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,62 @@
|
||||
/*
|
||||
* 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.cli;
|
||||
|
||||
import org.springframework.core.Ordered;
|
||||
import org.springframework.core.annotation.Order;
|
||||
import org.springframework.shell.plugin.support.DefaultBannerProvider;
|
||||
import org.springframework.shell.support.util.OsUtils;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
@Order(Ordered.HIGHEST_PRECEDENCE)
|
||||
public class HoodieSplashScreen extends DefaultBannerProvider {
|
||||
|
||||
static {
|
||||
System.out.println("HoodieSplashScreen loaded");
|
||||
}
|
||||
|
||||
private static String screen = "============================================" + OsUtils.LINE_SEPARATOR
|
||||
+ "* *" + OsUtils.LINE_SEPARATOR
|
||||
+ "* _ _ _ _ *" + OsUtils.LINE_SEPARATOR
|
||||
+ "* | | | | | | (_) *" + OsUtils.LINE_SEPARATOR
|
||||
+ "* | |__| | __| | - *" + OsUtils.LINE_SEPARATOR
|
||||
+ "* | __ || | / _` | || *" + OsUtils.LINE_SEPARATOR
|
||||
+ "* | | | || || (_| | || *" + OsUtils.LINE_SEPARATOR
|
||||
+ "* |_| |_|\\___/ \\____/ || *" + OsUtils.LINE_SEPARATOR
|
||||
+ "* *" + OsUtils.LINE_SEPARATOR
|
||||
+ "============================================" + OsUtils.LINE_SEPARATOR;
|
||||
|
||||
public String getBanner() {
|
||||
return screen;
|
||||
}
|
||||
|
||||
public String getVersion() {
|
||||
return "1.0";
|
||||
}
|
||||
|
||||
public String getWelcomeMessage() {
|
||||
return "Welcome to Hoodie CLI. Please type help if you are looking for help. ";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getProviderName() {
|
||||
return "Hoodie Banner";
|
||||
}
|
||||
}
|
||||
34
hudi-cli/src/main/java/org/apache/hudi/cli/Main.java
Normal file
34
hudi-cli/src/main/java/org/apache/hudi/cli/Main.java
Normal file
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* 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.cli;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.springframework.shell.Bootstrap;
|
||||
|
||||
public class Main {
|
||||
|
||||
/**
|
||||
* Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE
|
||||
*/
|
||||
public static void main(String[] args) throws IOException {
|
||||
System.out.println("Main called");
|
||||
new HoodieSplashScreen();
|
||||
Bootstrap.main(args);
|
||||
}
|
||||
}
|
||||
178
hudi-cli/src/main/java/org/apache/hudi/cli/Table.java
Normal file
178
hudi-cli/src/main/java/org/apache/hudi/cli/Table.java
Normal file
@@ -0,0 +1,178 @@
|
||||
/*
|
||||
* 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.cli;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Table to be rendered. This class takes care of ordering
|
||||
* rows and limiting before renderer renders it.
|
||||
*/
|
||||
public class Table implements Iterable<List<String>> {
|
||||
|
||||
// Header for this table
|
||||
private final TableHeader rowHeader;
|
||||
// User-specified conversions before rendering
|
||||
private final Map<String, Function<Object, String>> fieldNameToConverterMap;
|
||||
// Option attribute to track sorting field
|
||||
private final Option<String> orderingFieldNameOptional;
|
||||
// Whether sorting has to be in descending order (by default : optional)
|
||||
private final Option<Boolean> isDescendingOptional;
|
||||
// Limit the number of entries rendered
|
||||
private final Option<Integer> limitOptional;
|
||||
// Raw list of rows
|
||||
private final List<List<Comparable>> rawRows;
|
||||
// Flag to determine if all the rows have been added
|
||||
private boolean finishedAdding = false;
|
||||
// Rows ready for Rendering
|
||||
private List<List<String>> renderRows;
|
||||
|
||||
public Table(TableHeader rowHeader,
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap,
|
||||
Option<String> orderingFieldNameOptional,
|
||||
Option<Boolean> isDescendingOptional,
|
||||
Option<Integer> limitOptional) {
|
||||
this.rowHeader = rowHeader;
|
||||
this.fieldNameToConverterMap = fieldNameToConverterMap;
|
||||
this.orderingFieldNameOptional = orderingFieldNameOptional;
|
||||
this.isDescendingOptional = isDescendingOptional;
|
||||
this.limitOptional = limitOptional;
|
||||
this.rawRows = new ArrayList<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Main API to add row to the table
|
||||
* @param row Row
|
||||
*/
|
||||
public Table add(List<Comparable> row) {
|
||||
if (finishedAdding) {
|
||||
throw new IllegalStateException("Container already marked done for adding. No more entries can be added.");
|
||||
}
|
||||
|
||||
if (rowHeader.getFieldNames().size() != row.size()) {
|
||||
throw new IllegalArgumentException("Incorrect number of fields in row. Expected: "
|
||||
+ rowHeader.getFieldNames().size() + ", Got: " + row.size() + ", Row: " + row);
|
||||
}
|
||||
|
||||
this.rawRows.add(new ArrayList<>(row));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add all rows
|
||||
* @param rows Rows to be aded
|
||||
* @return
|
||||
*/
|
||||
public Table addAll(List<List<Comparable>> rows) {
|
||||
rows.forEach(r -> add(r));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add all rows
|
||||
* @param rows Rows to be added
|
||||
* @return
|
||||
*/
|
||||
public Table addAllRows(List<Comparable[]> rows) {
|
||||
rows.forEach(r -> add(Arrays.asList(r)));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* API to let the table know writing is over and reading is going to start
|
||||
*/
|
||||
public Table flip() {
|
||||
this.finishedAdding = true;
|
||||
sortAndLimit();
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sorting of rows by a specified field
|
||||
* @return
|
||||
*/
|
||||
private List<List<Comparable>> orderRows() {
|
||||
return orderingFieldNameOptional.map(orderingColumnName -> {
|
||||
return rawRows.stream().sorted(new Comparator<List<Comparable>>() {
|
||||
@Override
|
||||
public int compare(List<Comparable> row1, List<Comparable> row2) {
|
||||
Comparable fieldForRow1 = row1.get(rowHeader.indexOf(orderingColumnName));
|
||||
Comparable fieldForRow2 = row2.get(rowHeader.indexOf(orderingColumnName));
|
||||
int cmpRawResult = fieldForRow1.compareTo(fieldForRow2);
|
||||
return isDescendingOptional.map(isDescending -> {
|
||||
return isDescending ? -1 * cmpRawResult : cmpRawResult;
|
||||
}).orElse(cmpRawResult);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
}).orElse(rawRows);
|
||||
}
|
||||
|
||||
/**
|
||||
* Prepares for rendering. Rows are sorted and limited
|
||||
*/
|
||||
private void sortAndLimit() {
|
||||
this.renderRows = new ArrayList<>();
|
||||
final int limit = this.limitOptional.orElse(rawRows.size());
|
||||
final List<List<Comparable>> orderedRows = orderRows();
|
||||
renderRows = orderedRows.stream().limit(limit).map(row -> {
|
||||
return IntStream.range(0, rowHeader.getNumFields()).mapToObj(idx -> {
|
||||
String fieldName = rowHeader.get(idx);
|
||||
if (fieldNameToConverterMap.containsKey(fieldName)) {
|
||||
return fieldNameToConverterMap.get(fieldName).apply(row.get(idx));
|
||||
}
|
||||
Object v = row.get(idx);
|
||||
return v == null ? "null" : v.toString();
|
||||
}).collect(Collectors.toList());
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<String>> iterator() {
|
||||
if (!finishedAdding) {
|
||||
throw new IllegalStateException("Container must be flipped before reading the data");
|
||||
}
|
||||
return renderRows.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void forEach(Consumer<? super List<String>> action) {
|
||||
if (!finishedAdding) {
|
||||
throw new IllegalStateException("Container must be flipped before reading the data");
|
||||
}
|
||||
renderRows.forEach(action);
|
||||
}
|
||||
|
||||
public List<String> getFieldNames() {
|
||||
return rowHeader.getFieldNames();
|
||||
}
|
||||
|
||||
public List<List<String>> getRenderRows() {
|
||||
return renderRows;
|
||||
}
|
||||
}
|
||||
71
hudi-cli/src/main/java/org/apache/hudi/cli/TableHeader.java
Normal file
71
hudi-cli/src/main/java/org/apache/hudi/cli/TableHeader.java
Normal file
@@ -0,0 +1,71 @@
|
||||
/*
|
||||
* 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.cli;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Header for the table to be rendered
|
||||
*/
|
||||
public class TableHeader {
|
||||
|
||||
// List of fields (columns)
|
||||
private final List<String> fieldNames = new ArrayList<>();
|
||||
|
||||
/**
|
||||
* Add a field (column) to table
|
||||
*
|
||||
* @param fieldName field Name
|
||||
*/
|
||||
public TableHeader addTableHeaderField(String fieldName) {
|
||||
fieldNames.add(fieldName);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all field names
|
||||
*/
|
||||
public List<String> getFieldNames() {
|
||||
return fieldNames;
|
||||
}
|
||||
|
||||
/**
|
||||
* Index of the field in the table
|
||||
*
|
||||
* @param fieldName Field Name
|
||||
*/
|
||||
public int indexOf(String fieldName) {
|
||||
return fieldNames.indexOf(fieldName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup field by offset
|
||||
*/
|
||||
public String get(int index) {
|
||||
return fieldNames.get(index);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get number of fields in the table
|
||||
*/
|
||||
public int getNumFields() {
|
||||
return fieldNames.size();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,239 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.avro.specific.SpecificData;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class ArchivedCommitsCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"show archived commits"})
|
||||
public boolean isShowArchivedCommitAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "show archived commit stats", help = "Read commits from archived files and show details")
|
||||
public String showArchivedCommits(
|
||||
@CliOption(key = {"archiveFolderPattern"}, help = "Archive Folder", unspecifiedDefaultValue = "") String folder,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
System.out.println("===============> Showing only " + limit + " archived commits <===============");
|
||||
String basePath = HoodieCLI.tableMetadata.getBasePath();
|
||||
Path archivePath = new Path(basePath + "/.hoodie/.commits_.archive*");
|
||||
if (folder != null && !folder.isEmpty()) {
|
||||
archivePath = new Path(basePath + "/.hoodie/" + folder);
|
||||
}
|
||||
FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
|
||||
List<Comparable[]> allStats = new ArrayList<>();
|
||||
for (FileStatus fs : fsStatuses) {
|
||||
//read the archived file
|
||||
Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
|
||||
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
|
||||
|
||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||
//read the avro blocks
|
||||
while (reader.hasNext()) {
|
||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||
List<IndexedRecord> records = blk.getRecords();
|
||||
readRecords.addAll(records);
|
||||
}
|
||||
List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r)
|
||||
.filter(r -> r.get("actionType").toString().equals(HoodieTimeline.COMMIT_ACTION)
|
||||
|| r.get("actionType").toString().equals(HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.flatMap(r -> {
|
||||
HoodieCommitMetadata metadata =
|
||||
(HoodieCommitMetadata) SpecificData.get().deepCopy(HoodieCommitMetadata.SCHEMA$,
|
||||
r.get("hoodieCommitMetadata"));
|
||||
final String instantTime = r.get("commitTime").toString();
|
||||
final String action = r.get("actionType").toString();
|
||||
return metadata.getPartitionToWriteStats().values().stream().flatMap(hoodieWriteStats -> {
|
||||
return hoodieWriteStats.stream().map(hoodieWriteStat -> {
|
||||
List<Comparable> row = new ArrayList<>();
|
||||
row.add(action);
|
||||
row.add(instantTime);
|
||||
row.add(hoodieWriteStat.getPartitionPath());
|
||||
row.add(hoodieWriteStat.getFileId());
|
||||
row.add(hoodieWriteStat.getPrevCommit());
|
||||
row.add(hoodieWriteStat.getNumWrites());
|
||||
row.add(hoodieWriteStat.getNumInserts());
|
||||
row.add(hoodieWriteStat.getNumDeletes());
|
||||
row.add(hoodieWriteStat.getNumUpdateWrites());
|
||||
row.add(hoodieWriteStat.getTotalLogFiles());
|
||||
row.add(hoodieWriteStat.getTotalLogBlocks());
|
||||
row.add(hoodieWriteStat.getTotalCorruptLogBlock());
|
||||
row.add(hoodieWriteStat.getTotalRollbackBlocks());
|
||||
row.add(hoodieWriteStat.getTotalLogRecords());
|
||||
row.add(hoodieWriteStat.getTotalUpdatedRecordsCompacted());
|
||||
row.add(hoodieWriteStat.getTotalWriteBytes());
|
||||
row.add(hoodieWriteStat.getTotalWriteErrors());
|
||||
return row;
|
||||
});
|
||||
}).map(rowList -> rowList.toArray(new Comparable[0]));
|
||||
}).collect(Collectors.toList());
|
||||
allStats.addAll(readCommits);
|
||||
reader.close();
|
||||
}
|
||||
TableHeader header = new TableHeader().addTableHeaderField("action")
|
||||
.addTableHeaderField("instant")
|
||||
.addTableHeaderField("partition")
|
||||
.addTableHeaderField("file_id")
|
||||
.addTableHeaderField("prev_instant")
|
||||
.addTableHeaderField("num_writes")
|
||||
.addTableHeaderField("num_inserts")
|
||||
.addTableHeaderField("num_deletes")
|
||||
.addTableHeaderField("num_update_writes")
|
||||
.addTableHeaderField("total_log_files")
|
||||
.addTableHeaderField("total_log_blocks")
|
||||
.addTableHeaderField("total_corrupt_log_blocks")
|
||||
.addTableHeaderField("total_rollback_blocks")
|
||||
.addTableHeaderField("total_log_records")
|
||||
.addTableHeaderField("total_updated_records_compacted")
|
||||
.addTableHeaderField("total_write_bytes")
|
||||
.addTableHeaderField("total_write_errors");
|
||||
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, allStats);
|
||||
}
|
||||
|
||||
@CliCommand(value = "show archived commits", help = "Read commits from archived files and show details")
|
||||
public String showCommits(
|
||||
@CliOption(key = {"skipMetadata"}, help = "Skip displaying commit metadata", unspecifiedDefaultValue = "true")
|
||||
boolean skipMetadata,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
System.out.println("===============> Showing only " + limit + " archived commits <===============");
|
||||
String basePath = HoodieCLI.tableMetadata.getBasePath();
|
||||
FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf)
|
||||
.globStatus(new Path(basePath + "/.hoodie/.commits_.archive*"));
|
||||
List<Comparable[]> allCommits = new ArrayList<>();
|
||||
for (FileStatus fs : fsStatuses) {
|
||||
//read the archived file
|
||||
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
|
||||
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
|
||||
|
||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||
//read the avro blocks
|
||||
while (reader.hasNext()) {
|
||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||
List<IndexedRecord> records = blk.getRecords();
|
||||
readRecords.addAll(records);
|
||||
}
|
||||
List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r ->
|
||||
readCommit(r, skipMetadata))
|
||||
.collect(Collectors.toList());
|
||||
allCommits.addAll(readCommits);
|
||||
reader.close();
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader().addTableHeaderField("CommitTime")
|
||||
.addTableHeaderField("CommitType");
|
||||
|
||||
if (!skipMetadata) {
|
||||
header = header.addTableHeaderField("CommitDetails");
|
||||
}
|
||||
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, allCommits);
|
||||
}
|
||||
|
||||
private Comparable[] readCommit(GenericRecord record, boolean skipMetadata) {
|
||||
List<Object> commitDetails = new ArrayList<>();
|
||||
try {
|
||||
switch (record.get("actionType").toString()) {
|
||||
case HoodieTimeline.CLEAN_ACTION: {
|
||||
commitDetails.add(record.get("commitTime"));
|
||||
commitDetails.add(record.get("actionType").toString());
|
||||
if (!skipMetadata) {
|
||||
commitDetails.add(record.get("hoodieCleanMetadata").toString());
|
||||
}
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMMIT_ACTION: {
|
||||
commitDetails.add(record.get("commitTime"));
|
||||
commitDetails.add(record.get("actionType").toString());
|
||||
if (!skipMetadata) {
|
||||
commitDetails.add(record.get("hoodieCommitMetadata").toString());
|
||||
}
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
||||
commitDetails.add(record.get("commitTime"));
|
||||
commitDetails.add(record.get("actionType").toString());
|
||||
if (!skipMetadata) {
|
||||
commitDetails.add(record.get("hoodieCommitMetadata").toString());
|
||||
}
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.ROLLBACK_ACTION: {
|
||||
commitDetails.add(record.get("commitTime"));
|
||||
commitDetails.add(record.get("actionType").toString());
|
||||
if (!skipMetadata) {
|
||||
commitDetails.add(record.get("hoodieRollbackMetadata").toString());
|
||||
}
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.SAVEPOINT_ACTION: {
|
||||
commitDetails.add(record.get("commitTime"));
|
||||
commitDetails.add(record.get("actionType").toString());
|
||||
if (!skipMetadata) {
|
||||
commitDetails.add(record.get("hoodieSavePointMetadata").toString());
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
return commitDetails.toArray(new Comparable[commitDetails.size()]);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
return commitDetails.toArray(new Comparable[commitDetails.size()]);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,135 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class CleansCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"cleans show"})
|
||||
public boolean isShowAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"cleans refresh"})
|
||||
public boolean isRefreshAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"clean showpartitions"})
|
||||
public boolean isCommitShowAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "cleans show", help = "Show the cleans")
|
||||
public String showCleans(
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants();
|
||||
List<HoodieInstant> cleans = timeline.getInstants().collect(Collectors.toList());
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
Collections.reverse(cleans);
|
||||
for (int i = 0; i < cleans.size(); i++) {
|
||||
HoodieInstant clean = cleans.get(i);
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils
|
||||
.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get());
|
||||
rows.add(new Comparable[]{clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
|
||||
cleanMetadata.getTotalFilesDeleted(), cleanMetadata.getTimeTakenInMillis()});
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("CleanTime")
|
||||
.addTableHeaderField("EarliestCommandRetained")
|
||||
.addTableHeaderField("Total Files Deleted")
|
||||
.addTableHeaderField("Total Time Taken");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "cleans refresh", help = "Refresh the commits")
|
||||
public String refreshCleans() throws IOException {
|
||||
HoodieCLI.refreshTableMetadata();
|
||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " refreshed.";
|
||||
}
|
||||
|
||||
@CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean")
|
||||
public String showCleanPartitions(
|
||||
@CliOption(key = {"clean"}, help = "clean to show") final String commitTime,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants();
|
||||
HoodieInstant cleanInstant = new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(cleanInstant)) {
|
||||
return "Clean " + commitTime + " not found in metadata " + timeline;
|
||||
}
|
||||
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata(
|
||||
timeline.getInstantDetails(cleanInstant).get());
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
for (Map.Entry<String, HoodieCleanPartitionMetadata> entry : cleanMetadata.getPartitionMetadata().entrySet()) {
|
||||
String path = entry.getKey();
|
||||
HoodieCleanPartitionMetadata stats = entry.getValue();
|
||||
String policy = stats.getPolicy();
|
||||
Integer totalSuccessDeletedFiles = stats.getSuccessDeleteFiles().size();
|
||||
Integer totalFailedDeletedFiles = stats.getFailedDeleteFiles().size();
|
||||
rows.add(new Comparable[]{path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles});
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("Cleaning policy")
|
||||
.addTableHeaderField("Total Files Successfully Deleted")
|
||||
.addTableHeaderField("Total Failed Deletions");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,302 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.cli.utils.InputStreamConsumer;
|
||||
import org.apache.hudi.cli.utils.SparkUtil;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.NumericUtils;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class CommitsCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"commits show"})
|
||||
public boolean isShowAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commits refresh"})
|
||||
public boolean isRefreshAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commit rollback"})
|
||||
public boolean isRollbackAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commit show"})
|
||||
public boolean isCommitShowAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits show", help = "Show the commits")
|
||||
public String showCommits(@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
|
||||
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
Collections.reverse(commits);
|
||||
for (int i = 0; i < commits.size(); i++) {
|
||||
HoodieInstant commit = commits.get(i);
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
rows.add(new Comparable[]{commit.getTimestamp(),
|
||||
commitMetadata.fetchTotalBytesWritten(),
|
||||
commitMetadata.fetchTotalFilesInsert(),
|
||||
commitMetadata.fetchTotalFilesUpdated(),
|
||||
commitMetadata.fetchTotalPartitionsWritten(),
|
||||
commitMetadata.fetchTotalRecordsWritten(),
|
||||
commitMetadata.fetchTotalUpdateRecordsWritten(),
|
||||
commitMetadata.fetchTotalWriteErrors()});
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Total Bytes Written", entry -> {
|
||||
return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
});
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("CommitTime")
|
||||
.addTableHeaderField("Total Bytes Written")
|
||||
.addTableHeaderField("Total Files Added")
|
||||
.addTableHeaderField("Total Files Updated")
|
||||
.addTableHeaderField("Total Partitions Written")
|
||||
.addTableHeaderField("Total Records Written")
|
||||
.addTableHeaderField("Total Update Records Written")
|
||||
.addTableHeaderField("Total Errors");
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits refresh", help = "Refresh the commits")
|
||||
public String refreshCommits() throws IOException {
|
||||
HoodieCLI.refreshTableMetadata();
|
||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " refreshed.";
|
||||
}
|
||||
|
||||
@CliCommand(value = "commit rollback", help = "Rollback a commit")
|
||||
public String rollbackCommit(@CliOption(key = {"commit"}, help = "Commit to rollback") final String commitTime,
|
||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") final String sparkPropertiesPath)
|
||||
throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(commitInstant)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||
}
|
||||
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher
|
||||
.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), commitTime, HoodieCLI.tableMetadata.getBasePath());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
// Refresh the current
|
||||
refreshCommits();
|
||||
if (exitCode != 0) {
|
||||
return "Commit " + commitTime + " failed to roll back";
|
||||
}
|
||||
return "Commit " + commitTime + " rolled back";
|
||||
}
|
||||
|
||||
@CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit")
|
||||
public String showCommitPartitions(
|
||||
@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(commitInstant)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||
}
|
||||
HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats().entrySet()) {
|
||||
String path = entry.getKey();
|
||||
List<HoodieWriteStat> stats = entry.getValue();
|
||||
long totalFilesAdded = 0;
|
||||
long totalFilesUpdated = 0;
|
||||
long totalRecordsUpdated = 0;
|
||||
long totalRecordsInserted = 0;
|
||||
long totalBytesWritten = 0;
|
||||
long totalWriteErrors = 0;
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
if (stat.getPrevCommit().equals(HoodieWriteStat.NULL_COMMIT)) {
|
||||
totalFilesAdded += 1;
|
||||
totalRecordsInserted += stat.getNumWrites();
|
||||
} else {
|
||||
totalFilesUpdated += 1;
|
||||
totalRecordsUpdated += stat.getNumUpdateWrites();
|
||||
}
|
||||
totalBytesWritten += stat.getTotalWriteBytes();
|
||||
totalWriteErrors += stat.getTotalWriteErrors();
|
||||
}
|
||||
rows.add(new Comparable[]{path, totalFilesAdded, totalFilesUpdated,
|
||||
totalRecordsInserted, totalRecordsUpdated,
|
||||
totalBytesWritten, totalWriteErrors});
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Total Bytes Written", entry -> {
|
||||
return NumericUtils.humanReadableByteCount((Long.valueOf(entry.toString())));
|
||||
});
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("Total Files Added")
|
||||
.addTableHeaderField("Total Files Updated")
|
||||
.addTableHeaderField("Total Records Inserted")
|
||||
.addTableHeaderField("Total Records Updated")
|
||||
.addTableHeaderField("Total Bytes Written")
|
||||
.addTableHeaderField("Total Errors");
|
||||
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "commit showfiles", help = "Show file level details of a commit")
|
||||
public String showCommitFiles(
|
||||
@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(commitInstant)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||
}
|
||||
HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats().entrySet()) {
|
||||
String path = entry.getKey();
|
||||
List<HoodieWriteStat> stats = entry.getValue();
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
rows.add(new Comparable[]{path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(),
|
||||
stat.getNumWrites(), stat.getTotalWriteBytes(),
|
||||
stat.getTotalWriteErrors(),
|
||||
stat.getFileSizeInBytes()
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("File ID")
|
||||
.addTableHeaderField("Previous Commit")
|
||||
.addTableHeaderField("Total Records Updated")
|
||||
.addTableHeaderField("Total Records Written")
|
||||
.addTableHeaderField("Total Bytes Written")
|
||||
.addTableHeaderField("Total Errors")
|
||||
.addTableHeaderField("File Size");
|
||||
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commits compare"})
|
||||
public boolean isCompareCommitsAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset")
|
||||
public String compareCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path)
|
||||
throws Exception {
|
||||
|
||||
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path);
|
||||
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
||||
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
String targetLatestCommit =
|
||||
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp();
|
||||
String sourceLatestCommit =
|
||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp();
|
||||
|
||||
if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// source is behind the target
|
||||
List<String> commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
|
||||
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
return "Source " + source.getTableConfig().getTableName() + " is behind by " + commitsToCatchup.size()
|
||||
+ " commits. Commits to catch up - " + commitsToCatchup;
|
||||
} else {
|
||||
List<String> commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
|
||||
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
return "Source " + source.getTableConfig().getTableName() + " is ahead by " + commitsToCatchup.size()
|
||||
+ " commits. Commits to catch up - " + commitsToCatchup;
|
||||
}
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"commits sync"})
|
||||
public boolean isSyncCommitsAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset")
|
||||
public String syncCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path)
|
||||
throws Exception {
|
||||
HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path);
|
||||
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
|
||||
return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " and "
|
||||
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,512 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.CompactionAdminClient.RenameOpResult;
|
||||
import org.apache.hudi.CompactionAdminClient.ValidationOpResult;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.cli.commands.SparkMain.SparkCommand;
|
||||
import org.apache.hudi.cli.utils.InputStreamConsumer;
|
||||
import org.apache.hudi.cli.utils.SparkUtil;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
import org.apache.spark.util.Utils;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class CompactionCommand implements CommandMarker {
|
||||
|
||||
private static Logger log = LogManager.getLogger(CompactionCommand.class);
|
||||
|
||||
private static final String TMP_DIR = "/tmp/";
|
||||
|
||||
@CliAvailabilityIndicator({"compactions show all", "compaction show", "compaction run", "compaction schedule"})
|
||||
public boolean isAvailable() {
|
||||
return (HoodieCLI.tableMetadata != null)
|
||||
&& (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ);
|
||||
}
|
||||
|
||||
@CliCommand(value = "compactions show all", help = "Shows all compactions that are in active timeline")
|
||||
public String compactionsAll(
|
||||
@CliOption(key = {
|
||||
"includeExtraMetadata"}, help = "Include extra metadata", unspecifiedDefaultValue = "false") final
|
||||
boolean includeExtraMetadata,
|
||||
@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final
|
||||
boolean headerOnly) throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionTimeline();
|
||||
HoodieTimeline commitTimeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
Set<String> committed = commitTimeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
|
||||
|
||||
List<HoodieInstant> instants = timeline.getInstants().collect(Collectors.toList());
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
Collections.reverse(instants);
|
||||
for (int i = 0; i < instants.size(); i++) {
|
||||
HoodieInstant instant = instants.get(i);
|
||||
HoodieCompactionPlan workload = null;
|
||||
if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
try {
|
||||
// This could be a completed compaction. Assume a compaction request file is present but skip if fails
|
||||
workload = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
} catch (HoodieIOException ioe) {
|
||||
// SKIP
|
||||
}
|
||||
} else {
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
}
|
||||
|
||||
if (null != workload) {
|
||||
HoodieInstant.State state = instant.getState();
|
||||
if (committed.contains(instant.getTimestamp())) {
|
||||
state = State.COMPLETED;
|
||||
}
|
||||
if (includeExtraMetadata) {
|
||||
rows.add(new Comparable[]{instant.getTimestamp(),
|
||||
state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size(),
|
||||
workload.getExtraMetadata().toString()});
|
||||
} else {
|
||||
rows.add(new Comparable[]{instant.getTimestamp(),
|
||||
state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size()});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Compaction Instant Time")
|
||||
.addTableHeaderField("State")
|
||||
.addTableHeaderField("Total FileIds to be Compacted");
|
||||
if (includeExtraMetadata) {
|
||||
header = header.addTableHeaderField("Extra Metadata");
|
||||
}
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction show", help = "Shows compaction details for a specific compaction instant")
|
||||
public String compactionShow(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Base path for the target hoodie dataset") final
|
||||
String compactionInstantTime,
|
||||
@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
if ((null != workload) && (null != workload.getOperations())) {
|
||||
for (HoodieCompactionOperation op : workload.getOperations()) {
|
||||
rows.add(new Comparable[]{op.getPartitionPath(),
|
||||
op.getFileId(),
|
||||
op.getBaseInstantTime(),
|
||||
op.getDataFilePath(),
|
||||
op.getDeltaFilePaths().size(),
|
||||
op.getMetrics() == null ? "" : op.getMetrics().toString()
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Base Instant")
|
||||
.addTableHeaderField("Data File Path")
|
||||
.addTableHeaderField("Total Delta Files")
|
||||
.addTableHeaderField("getMetrics");
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction schedule", help = "Schedule Compaction")
|
||||
public String scheduleCompact(
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "1G", help = "Spark executor memory")
|
||||
final String sparkMemory) throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
// First get a compaction instant time and pass it to spark launcher for scheduling compaction
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_SCHEDULE.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
HoodieCLI.tableMetadata.getTableConfig().getTableName(), compactionInstantTime, sparkMemory);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to run compaction for " + compactionInstantTime;
|
||||
}
|
||||
return "Compaction successfully completed for " + compactionInstantTime;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction run", help = "Run Compaction for given instant time")
|
||||
public String compact(
|
||||
@CliOption(key = {"parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction")
|
||||
final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file")
|
||||
final String schemaFilePath,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G", help = "Spark executor memory")
|
||||
final String sparkMemory,
|
||||
@CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries")
|
||||
final String retry,
|
||||
@CliOption(key = "compactionInstant", mandatory = true, help = "Base path for the target hoodie dataset")
|
||||
final String compactionInstantTime) throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
HoodieCLI.tableMetadata.getTableConfig().getTableName(), compactionInstantTime, parallelism, schemaFilePath,
|
||||
sparkMemory, retry);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to run compaction for " + compactionInstantTime;
|
||||
}
|
||||
return "Compaction successfully completed for " + compactionInstantTime;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
private static String getTmpSerializerFile() {
|
||||
return TMP_DIR + UUID.randomUUID().toString() + ".ser";
|
||||
}
|
||||
|
||||
private <T> T deSerializeOperationResult(String inputP, FileSystem fs) throws Exception {
|
||||
Path inputPath = new Path(inputP);
|
||||
FSDataInputStream fsDataInputStream = fs.open(inputPath);
|
||||
ObjectInputStream in = new ObjectInputStream(fsDataInputStream);
|
||||
try {
|
||||
T result = (T) in.readObject();
|
||||
log.info("Result : " + result);
|
||||
return result;
|
||||
} finally {
|
||||
in.close();
|
||||
fsDataInputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction validate", help = "Validate Compaction")
|
||||
public String validateCompaction(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
|
||||
@CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
|
||||
@CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
String outputPathStr = getTmpSerializerFile();
|
||||
Path outputPath = new Path(outputPathStr);
|
||||
String output = null;
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_VALIDATE.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to validate compaction for " + compactionInstant;
|
||||
}
|
||||
List<ValidationOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
boolean valid = res.stream().map(r -> r.isSuccess()).reduce(Boolean::logicalAnd).orElse(true);
|
||||
String message = "\n\n\t COMPACTION PLAN " + (valid ? "VALID" : "INVALID") + "\n\n";
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
res.stream().forEach(r -> {
|
||||
Comparable[] row = new Comparable[]{r.getOperation().getFileId(),
|
||||
r.getOperation().getBaseInstantTime(),
|
||||
r.getOperation().getDataFilePath().isPresent() ? r.getOperation().getDataFilePath().get() : "",
|
||||
r.getOperation().getDeltaFilePaths().size(), r.isSuccess(),
|
||||
r.getException().isPresent() ? r.getException().get().getMessage() : ""};
|
||||
rows.add(row);
|
||||
});
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Base Instant Time")
|
||||
.addTableHeaderField("Base Data File")
|
||||
.addTableHeaderField("Num Delta Files")
|
||||
.addTableHeaderField("Valid")
|
||||
.addTableHeaderField("Error");
|
||||
|
||||
output = message + HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit,
|
||||
headerOnly, rows);
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
HoodieCLI.fs.delete(outputPath, false);
|
||||
}
|
||||
}
|
||||
return output;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction unschedule", help = "Unschedule Compaction")
|
||||
public String unscheduleCompaction(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
|
||||
@CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
|
||||
@CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
|
||||
@CliOption(key = {"skipValidation"}, help = "skip validation", unspecifiedDefaultValue = "false") boolean skipV,
|
||||
@CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
String outputPathStr = getTmpSerializerFile();
|
||||
Path outputPath = new Path(outputPathStr);
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_PLAN.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory, Boolean.valueOf(skipV).toString(), Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to unschedule compaction for " + compactionInstant;
|
||||
}
|
||||
List<RenameOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly,
|
||||
"unschedule pending compaction");
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
HoodieCLI.fs.delete(outputPath, false);
|
||||
}
|
||||
}
|
||||
return output;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction unscheduleFileId", help = "UnSchedule Compaction for a fileId")
|
||||
public String unscheduleCompactFile(
|
||||
@CliOption(key = "fileId", mandatory = true, help = "File Id") final String fileId,
|
||||
@CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
|
||||
@CliOption(key = {"skipValidation"}, help = "skip validation", unspecifiedDefaultValue = "false") boolean skipV,
|
||||
@CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {"headeronly"}, help = "Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
String outputPathStr = getTmpSerializerFile();
|
||||
Path outputPath = new Path(outputPathStr);
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_FILE.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), fileId, outputPathStr, "1", master,
|
||||
sparkMemory, Boolean.valueOf(skipV).toString(), Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to unschedule compaction for file " + fileId;
|
||||
}
|
||||
List<RenameOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly,
|
||||
"unschedule file from pending compaction");
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
HoodieCLI.fs.delete(outputPath, false);
|
||||
}
|
||||
}
|
||||
return output;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction repair", help = "Renames the files to make them consistent with the timeline as "
|
||||
+ "dictated by Hoodie metadata. Use when compaction unschedule fails partially.")
|
||||
public String repairCompaction(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
|
||||
@CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
|
||||
@CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
|
||||
@CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
String outputPathStr = getTmpSerializerFile();
|
||||
Path outputPath = new Path(outputPathStr);
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_REPAIR.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory, Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to unschedule compaction for " + compactionInstant;
|
||||
}
|
||||
List<RenameOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly, "repair compaction");
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
HoodieCLI.fs.delete(outputPath, false);
|
||||
}
|
||||
}
|
||||
return output;
|
||||
} else {
|
||||
throw new Exception("Compactions can only be run for table type : MERGE_ON_READ");
|
||||
}
|
||||
}
|
||||
|
||||
private String getRenamesToBePrinted(List<RenameOpResult> res, Integer limit,
|
||||
String sortByField, boolean descending, boolean headerOnly, String operation) {
|
||||
|
||||
Option<Boolean> result = Option.fromJavaOptional(
|
||||
res.stream().map(r -> r.isExecuted() && r.isSuccess()).reduce(Boolean::logicalAnd));
|
||||
if (result.isPresent()) {
|
||||
System.out.println("There were some file renames that needed to be done to " + operation);
|
||||
|
||||
if (result.get()) {
|
||||
System.out.println("All renames successfully completed to " + operation + " done !!");
|
||||
} else {
|
||||
System.out.println("Some renames failed. DataSet could be in inconsistent-state. "
|
||||
+ "Try running compaction repair");
|
||||
}
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
res.stream().forEach(r -> {
|
||||
Comparable[] row = new Comparable[] {
|
||||
r.getOperation().fileId, r.getOperation().srcPath, r.getOperation().destPath,
|
||||
r.isExecuted(), r.isSuccess(), r.getException().isPresent() ? r.getException().get().getMessage() : ""
|
||||
};
|
||||
rows.add(row);
|
||||
});
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Source File Path")
|
||||
.addTableHeaderField("Destination File Path")
|
||||
.addTableHeaderField("Rename Executed?")
|
||||
.addTableHeaderField("Rename Succeeded?")
|
||||
.addTableHeaderField("Error");
|
||||
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending,
|
||||
limit, headerOnly, rows);
|
||||
} else {
|
||||
return "No File renames needed to " + operation + ". Operation successful.";
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,132 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.exception.DatasetNotFoundException;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class DatasetsCommand implements CommandMarker {
|
||||
|
||||
static {
|
||||
System.out.println("DatasetsCommand getting loaded");
|
||||
}
|
||||
|
||||
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
|
||||
public String connect(
|
||||
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
|
||||
@CliOption(key = {"eventuallyConsistent"}, mandatory = false, unspecifiedDefaultValue = "false",
|
||||
help = "Enable eventual consistency") final boolean eventuallyConsistent,
|
||||
@CliOption(key = {"initialCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "2000",
|
||||
help = "Initial wait time for eventual consistency") final Integer initialConsistencyIntervalMs,
|
||||
@CliOption(key = {"maxCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "300000",
|
||||
help = "Max wait time for eventual consistency") final Integer maxConsistencyIntervalMs,
|
||||
@CliOption(key = {"maxCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "7",
|
||||
help = "Max checks for eventual consistency") final Integer maxConsistencyChecks) throws IOException {
|
||||
HoodieCLI.setConsistencyGuardConfig(
|
||||
ConsistencyGuardConfig.newBuilder()
|
||||
.withConsistencyCheckEnabled(eventuallyConsistent)
|
||||
.withInitialConsistencyCheckIntervalMs(initialConsistencyIntervalMs)
|
||||
.withMaxConsistencyCheckIntervalMs(maxConsistencyIntervalMs)
|
||||
.withMaxConsistencyChecks(maxConsistencyChecks)
|
||||
.build());
|
||||
HoodieCLI.initConf();
|
||||
HoodieCLI.connectTo(path);
|
||||
HoodieCLI.initFS(true);
|
||||
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
|
||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded";
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a Hoodie Table if it does not exist
|
||||
*
|
||||
* @param path Base Path
|
||||
* @param name Hoodie Table Name
|
||||
* @param tableTypeStr Hoodie Table Type
|
||||
* @param payloadClass Payload Class
|
||||
*/
|
||||
@CliCommand(value = "create", help = "Create a hoodie table if not present")
|
||||
public String createTable(
|
||||
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
|
||||
@CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name,
|
||||
@CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE",
|
||||
help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr,
|
||||
@CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.HoodieAvroPayload",
|
||||
help = "Payload Class") final String payloadClass) throws IOException {
|
||||
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
boolean existing = false;
|
||||
try {
|
||||
new HoodieTableMetaClient(HoodieCLI.conf, path);
|
||||
existing = true;
|
||||
} catch (DatasetNotFoundException dfe) {
|
||||
// expected
|
||||
}
|
||||
|
||||
// Do not touch table that already exist
|
||||
if (existing) {
|
||||
throw new IllegalStateException("Dataset already existing in path : " + path);
|
||||
}
|
||||
|
||||
final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr);
|
||||
HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, payloadClass);
|
||||
|
||||
// Now connect to ensure loading works
|
||||
return connect(path, false, 0, 0, 0);
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"desc"})
|
||||
public boolean isDescAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Describes table properties
|
||||
*/
|
||||
@CliCommand(value = "desc", help = "Describle Hoodie Table properties")
|
||||
public String descTable() {
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Property")
|
||||
.addTableHeaderField("Value");
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
rows.add(new Comparable[]{"basePath", HoodieCLI.tableMetadata.getBasePath()});
|
||||
rows.add(new Comparable[]{"metaPath", HoodieCLI.tableMetadata.getMetaPath()});
|
||||
rows.add(new Comparable[]{"fileSystem", HoodieCLI.tableMetadata.getFs().getScheme()});
|
||||
HoodieCLI.tableMetadata.getTableConfig().getProps().entrySet().forEach(e -> {
|
||||
rows.add(new Comparable[]{e.getKey(), e.getValue()});
|
||||
});
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,272 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.BiPredicate;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.NumericUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class FileSystemViewCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "show fsview all", help = "Show entire file-system view")
|
||||
public String showAllFileSlices(
|
||||
@CliOption(key = {"pathRegex"},
|
||||
help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") String globRegex,
|
||||
@CliOption(key = {"readOptimizedOnly"}, help = "Only display read-optimized view",
|
||||
unspecifiedDefaultValue = "false") boolean readOptimizedOnly,
|
||||
@CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed",
|
||||
unspecifiedDefaultValue = "") String maxInstant,
|
||||
@CliOption(key = {
|
||||
"includeMax"}, help = "Include Max Instant", unspecifiedDefaultValue = "false") boolean includeMaxInstant,
|
||||
@CliOption(key = {
|
||||
"includeInflight"}, help = "Include Inflight Instants", unspecifiedDefaultValue = "false")
|
||||
boolean includeInflight,
|
||||
@CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants", unspecifiedDefaultValue = "false")
|
||||
boolean excludeCompaction,
|
||||
@CliOption(key = {"limit"}, help = "Limit rows to be displayed", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieTableFileSystemView fsView = buildFileSystemView(globRegex, maxInstant, readOptimizedOnly, includeMaxInstant,
|
||||
includeInflight, excludeCompaction);
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
fsView.getAllFileGroups().forEach(fg -> fg.getAllFileSlices().forEach(fs -> {
|
||||
int idx = 0;
|
||||
// For ReadOptimized Views, do not display any delta-file related columns
|
||||
Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 8];
|
||||
row[idx++] = fg.getPartitionPath();
|
||||
row[idx++] = fg.getFileGroupId().getFileId();
|
||||
row[idx++] = fs.getBaseInstantTime();
|
||||
row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : "";
|
||||
row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1;
|
||||
if (!readOptimizedOnly) {
|
||||
row[idx++] = fs.getLogFiles().count();
|
||||
row[idx++] = fs.getLogFiles().mapToLong(lf -> lf.getFileSize()).sum();
|
||||
row[idx++] = fs.getLogFiles().collect(Collectors.toList()).toString();
|
||||
}
|
||||
rows.add(row);
|
||||
}));
|
||||
Function<Object, String> converterFunction =
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Total Delta File Size", converterFunction);
|
||||
fieldNameToConverterMap.put("Data-File Size", converterFunction);
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition")
|
||||
.addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant")
|
||||
.addTableHeaderField("Data-File")
|
||||
.addTableHeaderField("Data-File Size");
|
||||
if (!readOptimizedOnly) {
|
||||
header = header.addTableHeaderField("Num Delta Files")
|
||||
.addTableHeaderField("Total Delta File Size")
|
||||
.addTableHeaderField("Delta Files");
|
||||
}
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "show fsview latest", help = "Show latest file-system view")
|
||||
public String showLatestFileSlices(
|
||||
@CliOption(key = {"partitionPath"},
|
||||
help = "A valid paritition path", mandatory = true) String partition,
|
||||
@CliOption(key = {"readOptimizedOnly"}, help = "Only display read-optimized view",
|
||||
unspecifiedDefaultValue = "false") boolean readOptimizedOnly,
|
||||
@CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed",
|
||||
unspecifiedDefaultValue = "") String maxInstant,
|
||||
@CliOption(key = {"merge"}, help = "Merge File Slices due to pending compaction",
|
||||
unspecifiedDefaultValue = "true") final boolean merge,
|
||||
@CliOption(key = {"includeMax"}, help = "Include Max Instant", unspecifiedDefaultValue = "false")
|
||||
boolean includeMaxInstant,
|
||||
@CliOption(key = {"includeInflight"}, help = "Include Inflight Instants", unspecifiedDefaultValue = "false")
|
||||
boolean includeInflight,
|
||||
@CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants", unspecifiedDefaultValue = "false")
|
||||
boolean excludeCompaction,
|
||||
@CliOption(key = {"limit"}, help = "Limit rows to be displayed", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieTableFileSystemView fsView = buildFileSystemView(partition, maxInstant, readOptimizedOnly, includeMaxInstant,
|
||||
includeInflight, excludeCompaction);
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
|
||||
final Stream<FileSlice> fileSliceStream;
|
||||
if (!merge) {
|
||||
fileSliceStream = fsView.getLatestFileSlices(partition);
|
||||
} else {
|
||||
if (maxInstant.isEmpty()) {
|
||||
maxInstant = HoodieCLI.tableMetadata.getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant()
|
||||
.get().getTimestamp();
|
||||
}
|
||||
fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn(partition, maxInstant);
|
||||
}
|
||||
|
||||
fileSliceStream.forEach(fs -> {
|
||||
int idx = 0;
|
||||
Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 13];
|
||||
row[idx++] = partition;
|
||||
row[idx++] = fs.getFileId();
|
||||
row[idx++] = fs.getBaseInstantTime();
|
||||
row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : "";
|
||||
|
||||
long dataFileSize = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1;
|
||||
row[idx++] = dataFileSize;
|
||||
|
||||
if (!readOptimizedOnly) {
|
||||
row[idx++] = fs.getLogFiles().count();
|
||||
row[idx++] = fs.getLogFiles().mapToLong(lf -> lf.getFileSize()).sum();
|
||||
long logFilesScheduledForCompactionTotalSize = fs.getLogFiles()
|
||||
.filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.mapToLong(lf -> lf.getFileSize()).sum();
|
||||
row[idx++] = logFilesScheduledForCompactionTotalSize;
|
||||
|
||||
long logFilesUnscheduledTotalSize = fs.getLogFiles()
|
||||
.filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.mapToLong(lf -> lf.getFileSize()).sum();
|
||||
row[idx++] = logFilesUnscheduledTotalSize;
|
||||
|
||||
double logSelectedForCompactionToBaseRatio =
|
||||
dataFileSize > 0 ? logFilesScheduledForCompactionTotalSize / (dataFileSize * 1.0) : -1;
|
||||
row[idx++] = logSelectedForCompactionToBaseRatio;
|
||||
double logUnscheduledToBaseRatio =
|
||||
dataFileSize > 0 ? logFilesUnscheduledTotalSize / (dataFileSize * 1.0) : -1;
|
||||
row[idx++] = logUnscheduledToBaseRatio;
|
||||
|
||||
row[idx++] = fs.getLogFiles()
|
||||
.filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
row[idx++] = fs.getLogFiles()
|
||||
.filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
}
|
||||
rows.add(row);
|
||||
});
|
||||
|
||||
Function<Object, String> converterFunction =
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Data-File Size", converterFunction);
|
||||
if (!readOptimizedOnly) {
|
||||
fieldNameToConverterMap.put("Total Delta Size", converterFunction);
|
||||
fieldNameToConverterMap.put("Delta Size - compaction scheduled", converterFunction);
|
||||
fieldNameToConverterMap.put("Delta Size - compaction unscheduled", converterFunction);
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition")
|
||||
.addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant")
|
||||
.addTableHeaderField("Data-File")
|
||||
.addTableHeaderField("Data-File Size");
|
||||
|
||||
if (!readOptimizedOnly) {
|
||||
header = header.addTableHeaderField("Num Delta Files")
|
||||
.addTableHeaderField("Total Delta Size")
|
||||
.addTableHeaderField("Delta Size - compaction scheduled")
|
||||
.addTableHeaderField("Delta Size - compaction unscheduled")
|
||||
.addTableHeaderField("Delta To Base Ratio - compaction scheduled")
|
||||
.addTableHeaderField("Delta To Base Ratio - compaction unscheduled")
|
||||
.addTableHeaderField("Delta Files - compaction scheduled")
|
||||
.addTableHeaderField("Delta Files - compaction unscheduled");
|
||||
}
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build File System View
|
||||
* @param globRegex Path Regex
|
||||
* @param maxInstant Max Instants to be used for displaying file-instants
|
||||
* @param readOptimizedOnly Include only read optimized view
|
||||
* @param includeMaxInstant Include Max instant
|
||||
* @param includeInflight Include inflight instants
|
||||
* @param excludeCompaction Exclude Compaction instants
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private HoodieTableFileSystemView buildFileSystemView(String globRegex, String maxInstant, boolean readOptimizedOnly,
|
||||
boolean includeMaxInstant, boolean includeInflight, boolean excludeCompaction) throws IOException {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieCLI.tableMetadata.getHadoopConf(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), true);
|
||||
FileSystem fs = HoodieCLI.fs;
|
||||
String globPath = String.format("%s/%s/*", HoodieCLI.tableMetadata.getBasePath(), globRegex);
|
||||
FileStatus[] statuses = fs.globStatus(new Path(globPath));
|
||||
Stream<HoodieInstant> instantsStream = null;
|
||||
|
||||
HoodieTimeline timeline = null;
|
||||
if (readOptimizedOnly) {
|
||||
timeline = metaClient.getActiveTimeline().getCommitTimeline();
|
||||
} else if (excludeCompaction) {
|
||||
timeline = metaClient.getActiveTimeline().getCommitsTimeline();
|
||||
} else {
|
||||
timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline();
|
||||
}
|
||||
|
||||
if (!includeInflight) {
|
||||
timeline = timeline.filterCompletedInstants();
|
||||
}
|
||||
|
||||
instantsStream = timeline.getInstants();
|
||||
|
||||
if (!maxInstant.isEmpty()) {
|
||||
final BiPredicate<String, String> predicate;
|
||||
if (includeMaxInstant) {
|
||||
predicate = HoodieTimeline.GREATER_OR_EQUAL;
|
||||
} else {
|
||||
predicate = HoodieTimeline.GREATER;
|
||||
}
|
||||
instantsStream = instantsStream.filter(is -> predicate.test(maxInstant, is.getTimestamp()));
|
||||
}
|
||||
|
||||
HoodieTimeline filteredTimeline = new HoodieDefaultTimeline(instantsStream,
|
||||
(Function<HoodieInstant, Option<byte[]>> & Serializable) metaClient.getActiveTimeline()::getInstantDetails);
|
||||
return new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,85 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.commands.SparkMain.SparkCommand;
|
||||
import org.apache.hudi.cli.utils.InputStreamConsumer;
|
||||
import org.apache.hudi.cli.utils.SparkUtil;
|
||||
import org.apache.hudi.utilities.HDFSParquetImporter.FormatValidator;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
import org.apache.spark.util.Utils;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
@Component
|
||||
public class HDFSParquetImportCommand implements CommandMarker {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class);
|
||||
|
||||
@CliCommand(value = "hdfsparquetimport", help = "Imports Parquet dataset to a hoodie dataset")
|
||||
public String convert(
|
||||
@CliOption(key = "upsert", mandatory = false, unspecifiedDefaultValue = "false",
|
||||
help = "Uses upsert API instead of the default insert API of WriteClient") boolean useUpsert,
|
||||
@CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") final String srcPath,
|
||||
@CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") final String
|
||||
targetPath,
|
||||
@CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName,
|
||||
@CliOption(key = "tableType", mandatory = true, help = "Table type") final String tableType,
|
||||
@CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField,
|
||||
@CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name") final String
|
||||
partitionPathField,
|
||||
@CliOption(key = {
|
||||
"parallelism"}, mandatory = true, help = "Parallelism for hoodie insert") final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String
|
||||
schemaFilePath,
|
||||
@CliOption(key = "format", mandatory = true, help = "Format for the input data") final String format,
|
||||
@CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory,
|
||||
@CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception {
|
||||
|
||||
(new FormatValidator()).validate("format", format);
|
||||
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
|
||||
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
|
||||
String cmd = SparkCommand.IMPORT.toString();
|
||||
if (useUpsert) {
|
||||
cmd = SparkCommand.UPSERT.toString();
|
||||
}
|
||||
|
||||
sparkLauncher.addAppArgs(cmd, srcPath, targetPath, tableName, tableType, rowKeyField,
|
||||
partitionPathField, parallelism, schemaFilePath, sparkMemory, retry);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
if (exitCode != 0) {
|
||||
return "Failed to import dataset to hoodie format";
|
||||
}
|
||||
return "Dataset imported to hoodie format";
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,244 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import com.beust.jcommander.internal.Maps;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
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.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCorruptBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieMemoryConfig;
|
||||
import org.apache.hudi.hive.util.SchemaUtil;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
import parquet.avro.AvroSchemaConverter;
|
||||
import scala.Tuple2;
|
||||
import scala.Tuple3;
|
||||
|
||||
@Component
|
||||
public class HoodieLogFileCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"show logfiles"})
|
||||
public boolean isShowArchivedLogFileAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "show logfile metadata", help = "Read commit metadata from log files")
|
||||
public String showLogFileCommits(
|
||||
@CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified path for the log file") final
|
||||
String logFilePathPattern,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false")
|
||||
final boolean headerOnly) throws IOException {
|
||||
|
||||
FileSystem fs = HoodieCLI.tableMetadata.getFs();
|
||||
List<String> logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern)))
|
||||
.map(status -> status.getPath().toString()).collect(Collectors.toList());
|
||||
Map<String, List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType,
|
||||
String>>, Integer>>>
|
||||
commitCountAndMetadata = Maps.newHashMap();
|
||||
int totalEntries = 0;
|
||||
int numCorruptBlocks = 0;
|
||||
int dummyInstantTimeCount = 0;
|
||||
|
||||
for (String logFilePath : logFilePaths) {
|
||||
FileStatus[] fsStatus = fs.listStatus(new Path(logFilePath));
|
||||
Schema writerSchema = new AvroSchemaConverter().convert(
|
||||
SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath)));
|
||||
Reader reader = HoodieLogFormat
|
||||
.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema);
|
||||
|
||||
// read the avro blocks
|
||||
while (reader.hasNext()) {
|
||||
HoodieLogBlock n = reader.next();
|
||||
String instantTime;
|
||||
int recordCount = 0;
|
||||
if (n instanceof HoodieCorruptBlock) {
|
||||
try {
|
||||
instantTime = n.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME);
|
||||
if (instantTime == null) {
|
||||
throw new Exception("Invalid instant time " + instantTime);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
numCorruptBlocks++;
|
||||
instantTime = "corrupt_block_" + numCorruptBlocks;
|
||||
// could not read metadata for corrupt block
|
||||
}
|
||||
} else {
|
||||
instantTime = n.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME);
|
||||
if (instantTime == null) {
|
||||
// This can happen when reading archived commit files since they were written without any instant time
|
||||
dummyInstantTimeCount++;
|
||||
instantTime = "dummy_instant_time_" + dummyInstantTimeCount;
|
||||
}
|
||||
if (n instanceof HoodieAvroDataBlock) {
|
||||
recordCount = ((HoodieAvroDataBlock) n).getRecords().size();
|
||||
}
|
||||
}
|
||||
if (commitCountAndMetadata.containsKey(instantTime)) {
|
||||
commitCountAndMetadata.get(instantTime).add(
|
||||
new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount));
|
||||
totalEntries++;
|
||||
} else {
|
||||
List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>,
|
||||
Integer>> list = new ArrayList<>();
|
||||
list.add(
|
||||
new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount));
|
||||
commitCountAndMetadata.put(instantTime, list);
|
||||
totalEntries++;
|
||||
}
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
int i = 0;
|
||||
ObjectMapper objectMapper = new ObjectMapper();
|
||||
for (Map.Entry<String, List<Tuple3<HoodieLogBlockType,
|
||||
Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer>>> entry
|
||||
: commitCountAndMetadata.entrySet()) {
|
||||
String instantTime = entry.getKey().toString();
|
||||
for (Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>,
|
||||
Map<HeaderMetadataType, String>>, Integer> tuple3 : entry.getValue()) {
|
||||
Comparable[] output = new Comparable[5];
|
||||
output[0] = instantTime;
|
||||
output[1] = tuple3._3();
|
||||
output[2] = tuple3._1().toString();
|
||||
output[3] = objectMapper.writeValueAsString(tuple3._2()._1());
|
||||
output[4] = objectMapper.writeValueAsString(tuple3._2()._2());
|
||||
rows.add(output);
|
||||
i++;
|
||||
}
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("InstantTime")
|
||||
.addTableHeaderField("RecordCount")
|
||||
.addTableHeaderField("BlockType")
|
||||
.addTableHeaderField("HeaderMetadata")
|
||||
.addTableHeaderField("FooterMetadata");
|
||||
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "show logfile records", help = "Read records from log files")
|
||||
public String showLogFileRecords(@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit,
|
||||
@CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified paths for the log files")
|
||||
final String logFilePathPattern,
|
||||
@CliOption(key = "mergeRecords", mandatory = false, help = "If the records in the log files should be merged",
|
||||
unspecifiedDefaultValue = "false") final Boolean shouldMerge)
|
||||
throws IOException {
|
||||
|
||||
System.out.println("===============> Showing only " + limit + " records <===============");
|
||||
|
||||
FileSystem fs = HoodieCLI.tableMetadata.getFs();
|
||||
List<String> logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern)))
|
||||
.map(status -> status.getPath().toString()).collect(Collectors.toList());
|
||||
|
||||
// TODO : readerSchema can change across blocks/log files, fix this inside Scanner
|
||||
AvroSchemaConverter converter = new AvroSchemaConverter();
|
||||
// get schema from last log file
|
||||
Schema readerSchema = converter.convert(
|
||||
SchemaUtil.readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1))));
|
||||
|
||||
List<IndexedRecord> allRecords = new ArrayList<>();
|
||||
|
||||
if (shouldMerge) {
|
||||
System.out.println("===========================> MERGING RECORDS <===================");
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
|
||||
HoodieCLI.tableMetadata.getBasePath(), logFilePaths, readerSchema,
|
||||
HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get()
|
||||
.getTimestamp(),
|
||||
Long.valueOf(HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES),
|
||||
Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED),
|
||||
Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED),
|
||||
Integer.valueOf(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
|
||||
HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH);
|
||||
for (HoodieRecord<? extends HoodieRecordPayload> hoodieRecord : scanner) {
|
||||
Option<IndexedRecord> record = hoodieRecord.getData().getInsertValue(readerSchema);
|
||||
if (allRecords.size() >= limit) {
|
||||
break;
|
||||
}
|
||||
allRecords.add(record.get());
|
||||
}
|
||||
} else {
|
||||
for (String logFile : logFilePaths) {
|
||||
Schema writerSchema = new AvroSchemaConverter().convert(
|
||||
SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFile)));
|
||||
HoodieLogFormat.Reader reader = HoodieLogFormat
|
||||
.newReader(fs, new HoodieLogFile(new Path(logFile)), writerSchema);
|
||||
// read the avro blocks
|
||||
while (reader.hasNext()) {
|
||||
HoodieLogBlock n = reader.next();
|
||||
if (n instanceof HoodieAvroDataBlock) {
|
||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) n;
|
||||
List<IndexedRecord> records = blk.getRecords();
|
||||
allRecords.addAll(records);
|
||||
if (allRecords.size() >= limit) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
reader.close();
|
||||
if (allRecords.size() >= limit) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
String[][] rows = new String[allRecords.size() + 1][];
|
||||
int i = 0;
|
||||
for (IndexedRecord record : allRecords) {
|
||||
String[] data = new String[1];
|
||||
data[0] = record.toString();
|
||||
rows[i] = data;
|
||||
i++;
|
||||
}
|
||||
return HoodiePrintHelper.print(new String[] {"Records"}, rows);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,112 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.utils.CommitUtil;
|
||||
import org.apache.hudi.cli.utils.HiveUtil;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class HoodieSyncCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"sync validate"})
|
||||
public boolean isSyncVerificationAvailable() {
|
||||
return HoodieCLI.tableMetadata != null && HoodieCLI.syncTableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "sync validate", help = "Validate the sync by counting the number of records")
|
||||
public String validateSync(
|
||||
@CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode") final String mode,
|
||||
@CliOption(key = {"sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database") final String srcDb,
|
||||
@CliOption(key = {
|
||||
"targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database") final String tgtDb,
|
||||
@CliOption(key = {
|
||||
"partitionCount"}, unspecifiedDefaultValue = "5", help = "total number of recent partitions to validate")
|
||||
final int partitionCount,
|
||||
@CliOption(key = {
|
||||
"hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to") final String hiveServerUrl,
|
||||
@CliOption(key = {
|
||||
"hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to") final
|
||||
String hiveUser,
|
||||
@CliOption(key = {
|
||||
"hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") final
|
||||
String hivePass)
|
||||
throws Exception {
|
||||
HoodieTableMetaClient target = HoodieCLI.syncTableMetadata;
|
||||
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline();
|
||||
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
||||
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline();
|
||||
long sourceCount = 0;
|
||||
long targetCount = 0;
|
||||
if ("complete".equals(mode)) {
|
||||
sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass);
|
||||
targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass);
|
||||
} else if ("latestPartitions".equals(mode)) {
|
||||
sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass);
|
||||
targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass);
|
||||
}
|
||||
|
||||
String targetLatestCommit =
|
||||
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp();
|
||||
String sourceLatestCommit =
|
||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp();
|
||||
|
||||
if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// source is behind the target
|
||||
List<HoodieInstant> commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
|
||||
.getInstants().collect(Collectors.toList());
|
||||
if (commitsToCatchup.isEmpty()) {
|
||||
return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count("
|
||||
+ source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount);
|
||||
} else {
|
||||
long newInserts = CommitUtil.countNewRecords(target,
|
||||
commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()));
|
||||
return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count("
|
||||
+ source.getTableConfig().getTableName()
|
||||
+ ") == " + (targetCount - sourceCount) + ". Catch up count is " + newInserts;
|
||||
}
|
||||
} else {
|
||||
List<HoodieInstant> commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
|
||||
.getInstants().collect(Collectors.toList());
|
||||
if (commitsToCatchup.isEmpty()) {
|
||||
return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count("
|
||||
+ target.getTableConfig().getTableName() + ") == " + (sourceCount - targetCount);
|
||||
} else {
|
||||
long newInserts = CommitUtil.countNewRecords(source,
|
||||
commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()));
|
||||
return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count("
|
||||
+ target.getTableConfig().getTableName()
|
||||
+ ") == " + (sourceCount - targetCount) + ". Catch up count is " + newInserts;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,107 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.utils.InputStreamConsumer;
|
||||
import org.apache.hudi.cli.utils.SparkUtil;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class RepairsCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"repair deduplicate"})
|
||||
public boolean isRepairDeduplicateAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"repair addpartitionmeta"})
|
||||
public boolean isRepairAddPartitionMetaAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "repair deduplicate", help = "De-duplicate a partition path contains duplicates & produce "
|
||||
+ "repaired files to replace with")
|
||||
public String deduplicate(@CliOption(key = {
|
||||
"duplicatedPartitionPath"}, help = "Partition Path containing the duplicates", mandatory = true) final String
|
||||
duplicatedPartitionPath,
|
||||
@CliOption(key = {
|
||||
"repairedOutputPath"}, help = "Location to place the repaired files", mandatory = true) final String
|
||||
repairedOutputPath,
|
||||
@CliOption(key = {
|
||||
"sparkProperties"}, help = "Spark Properites File Path", mandatory = true) final String sparkPropertiesPath)
|
||||
throws Exception {
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath, repairedOutputPath,
|
||||
HoodieCLI.tableMetadata.getBasePath());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
|
||||
if (exitCode != 0) {
|
||||
return "Deduplicated files placed in: " + repairedOutputPath;
|
||||
}
|
||||
return "Deduplication failed ";
|
||||
}
|
||||
|
||||
|
||||
@CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present")
|
||||
public String addPartitionMeta(@CliOption(key = {
|
||||
"dryrun"}, help = "Should we actually add or just print what would be done", unspecifiedDefaultValue = "true")
|
||||
final boolean dryRun) throws IOException {
|
||||
|
||||
String latestCommit = HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get()
|
||||
.getTimestamp();
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionFoldersThreeLevelsDown(HoodieCLI.fs,
|
||||
HoodieCLI.tableMetadata.getBasePath());
|
||||
Path basePath = new Path(HoodieCLI.tableMetadata.getBasePath());
|
||||
String[][] rows = new String[partitionPaths.size() + 1][];
|
||||
|
||||
int ind = 0;
|
||||
for (String partition : partitionPaths) {
|
||||
Path partitionPath = FSUtils.getPartitionPath(basePath, partition);
|
||||
String[] row = new String[3];
|
||||
row[0] = partition;
|
||||
row[1] = "Yes";
|
||||
row[2] = "None";
|
||||
if (!HoodiePartitionMetadata.hasPartitionMetadata(HoodieCLI.fs, partitionPath)) {
|
||||
row[1] = "No";
|
||||
if (!dryRun) {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath,
|
||||
partitionPath);
|
||||
partitionMetadata.trySave(0);
|
||||
}
|
||||
}
|
||||
rows[ind++] = row;
|
||||
}
|
||||
|
||||
return HoodiePrintHelper.print(new String[] {"Partition Path", "Metadata Present?", "Action"}, rows);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,132 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTimeline.ROLLBACK_ACTION;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class RollbacksCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "show rollbacks", help = "List all rollback instants")
|
||||
public String showRollbacks(
|
||||
@CliOption(key = {"limit"}, help = "Limit #rows to be displayed", unspecifiedDefaultValue = "10") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = new RollbackTimeline(HoodieCLI.tableMetadata);
|
||||
HoodieTimeline rollback = activeTimeline.getRollbackTimeline().filterCompletedInstants();
|
||||
|
||||
final List<Comparable[]> rows = new ArrayList<>();
|
||||
rollback.getInstants().forEach(instant -> {
|
||||
try {
|
||||
HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata(
|
||||
activeTimeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class);
|
||||
metadata.getCommitsRollback().forEach(c -> {
|
||||
Comparable[] row = new Comparable[5];
|
||||
row[0] = metadata.getStartRollbackTime();
|
||||
row[1] = c;
|
||||
row[2] = metadata.getTotalFilesDeleted();
|
||||
row[3] = metadata.getTimeTakenInMillis();
|
||||
row[4] = metadata.getPartitionMetadata() != null ? metadata.getPartitionMetadata().size() : 0;
|
||||
rows.add(row);
|
||||
});
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
});
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Instant")
|
||||
.addTableHeaderField("Rolledback Instant")
|
||||
.addTableHeaderField("Total Files Deleted")
|
||||
.addTableHeaderField("Time taken in millis")
|
||||
.addTableHeaderField("Total Partitions");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "show rollback", help = "Show details of a rollback instant")
|
||||
public String showRollback(
|
||||
@CliOption(key = {"instant"}, help = "Rollback instant", mandatory = true) String rollbackInstant,
|
||||
@CliOption(key = {"limit"}, help = "Limit #rows to be displayed", unspecifiedDefaultValue = "10") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = new RollbackTimeline(HoodieCLI.tableMetadata);
|
||||
final List<Comparable[]> rows = new ArrayList<>();
|
||||
HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata(
|
||||
activeTimeline.getInstantDetails(new HoodieInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant))
|
||||
.get(), HoodieRollbackMetadata.class);
|
||||
metadata.getPartitionMetadata().entrySet().forEach(e -> {
|
||||
Stream.concat(e.getValue().getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)),
|
||||
e.getValue().getFailedDeleteFiles().stream().map(f -> Pair.of(f, false)))
|
||||
.forEach(fileWithDeleteStatus -> {
|
||||
Comparable[] row = new Comparable[5];
|
||||
row[0] = metadata.getStartRollbackTime();
|
||||
row[1] = metadata.getCommitsRollback().toString();
|
||||
row[2] = e.getKey();
|
||||
row[3] = fileWithDeleteStatus.getLeft();
|
||||
row[4] = fileWithDeleteStatus.getRight();
|
||||
rows.add(row);
|
||||
});
|
||||
});
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Instant")
|
||||
.addTableHeaderField("Rolledback Instants")
|
||||
.addTableHeaderField("Partition")
|
||||
.addTableHeaderField("Deleted File")
|
||||
.addTableHeaderField("Succeeded");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
/**
|
||||
* An Active timeline containing only rollbacks
|
||||
*/
|
||||
class RollbackTimeline extends HoodieActiveTimeline {
|
||||
|
||||
public RollbackTimeline(HoodieTableMetaClient metaClient) {
|
||||
super(metaClient, ImmutableSet.<String>builder().add(HoodieTimeline.ROLLBACK_EXTENSION).build());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,146 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.utils.InputStreamConsumer;
|
||||
import org.apache.hudi.cli.utils.SparkUtil;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class SavepointsCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"savepoints show"})
|
||||
public boolean isShowAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"savepoints refresh"})
|
||||
public boolean isRefreshAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
|
||||
@CliAvailabilityIndicator({"savepoint create"})
|
||||
public boolean isCreateSavepointAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"savepoint rollback"})
|
||||
public boolean isRollbackToSavepointAvailable() {
|
||||
return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline().getSavePointTimeline()
|
||||
.filterCompletedInstants().empty();
|
||||
}
|
||||
|
||||
@CliCommand(value = "savepoints show", help = "Show the savepoints")
|
||||
public String showSavepoints() throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getSavePointTimeline().filterCompletedInstants();
|
||||
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
|
||||
String[][] rows = new String[commits.size()][];
|
||||
Collections.reverse(commits);
|
||||
for (int i = 0; i < commits.size(); i++) {
|
||||
HoodieInstant commit = commits.get(i);
|
||||
rows[i] = new String[] {commit.getTimestamp()};
|
||||
}
|
||||
return HoodiePrintHelper.print(new String[] {"SavepointTime"}, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "savepoint create", help = "Savepoint a commit")
|
||||
public String savepoint(@CliOption(key = {"commit"}, help = "Commit to savepoint") final String commitTime,
|
||||
@CliOption(key = {"user"}, help = "User who is creating the savepoint") final String user,
|
||||
@CliOption(key = {"comments"}, help = "Comments for creating the savepoint") final String comments)
|
||||
throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(commitInstant)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||
}
|
||||
|
||||
HoodieWriteClient client = createHoodieClient(null, HoodieCLI.tableMetadata.getBasePath());
|
||||
if (client.savepoint(commitTime, user, comments)) {
|
||||
// Refresh the current
|
||||
refreshMetaClient();
|
||||
return String.format("The commit \"%s\" has been savepointed.", commitTime);
|
||||
}
|
||||
return String.format("Failed: Could not savepoint commit \"%s\".", commitTime);
|
||||
}
|
||||
|
||||
@CliCommand(value = "savepoint rollback", help = "Savepoint a commit")
|
||||
public String rollbackToSavepoint(
|
||||
@CliOption(key = {"savepoint"}, help = "Savepoint to rollback") final String commitTime,
|
||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") final String sparkPropertiesPath)
|
||||
throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(commitInstant)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||
}
|
||||
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(), commitTime,
|
||||
HoodieCLI.tableMetadata.getBasePath());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
// Refresh the current
|
||||
refreshMetaClient();
|
||||
if (exitCode != 0) {
|
||||
return "Savepoint " + commitTime + " failed to roll back";
|
||||
}
|
||||
return "Savepoint " + commitTime + " rolled back";
|
||||
}
|
||||
|
||||
|
||||
@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints")
|
||||
public String refreshMetaClient() throws IOException {
|
||||
HoodieCLI.refreshTableMetadata();
|
||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " refreshed.";
|
||||
}
|
||||
|
||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
return new HoodieWriteClient(jsc, config, false);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -0,0 +1,251 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.cli.DedupeSparkJob;
|
||||
import org.apache.hudi.cli.utils.SparkUtil;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.compact.strategy.UnBoundedCompactionStrategy;
|
||||
import org.apache.hudi.utilities.HDFSParquetImporter;
|
||||
import org.apache.hudi.utilities.HDFSParquetImporter.Config;
|
||||
import org.apache.hudi.utilities.HoodieCompactionAdminTool;
|
||||
import org.apache.hudi.utilities.HoodieCompactionAdminTool.Operation;
|
||||
import org.apache.hudi.utilities.HoodieCompactor;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
|
||||
public class SparkMain {
|
||||
|
||||
protected static final Logger LOG = Logger.getLogger(SparkMain.class);
|
||||
|
||||
/**
|
||||
* Commands
|
||||
*/
|
||||
enum SparkCommand {
|
||||
ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN,
|
||||
COMPACT_UNSCHEDULE_PLAN, COMPACT_UNSCHEDULE_FILE, COMPACT_VALIDATE, COMPACT_REPAIR
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
String command = args[0];
|
||||
LOG.info("Invoking SparkMain:" + command);
|
||||
|
||||
SparkCommand cmd = SparkCommand.valueOf(command);
|
||||
|
||||
JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + command);
|
||||
int returnCode = 0;
|
||||
switch (cmd) {
|
||||
case ROLLBACK:
|
||||
assert (args.length == 3);
|
||||
returnCode = rollback(jsc, args[1], args[2]);
|
||||
break;
|
||||
case DEDUPLICATE:
|
||||
assert (args.length == 4);
|
||||
returnCode = deduplicatePartitionPath(jsc, args[1], args[2], args[3]);
|
||||
break;
|
||||
case ROLLBACK_TO_SAVEPOINT:
|
||||
assert (args.length == 3);
|
||||
returnCode = rollbackToSavepoint(jsc, args[1], args[2]);
|
||||
break;
|
||||
case IMPORT:
|
||||
case UPSERT:
|
||||
assert (args.length == 11);
|
||||
returnCode = dataLoad(jsc, command, args[1], args[2], args[3], args[4], args[5], args[6],
|
||||
Integer.parseInt(args[7]), args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], Integer.parseInt(args[10]));
|
||||
break;
|
||||
case COMPACT_RUN:
|
||||
assert (args.length == 8);
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]),
|
||||
args[5], args[6], Integer.parseInt(args[7]), false);
|
||||
break;
|
||||
case COMPACT_SCHEDULE:
|
||||
assert (args.length == 5);
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], 1,
|
||||
"", args[4], 0, true);
|
||||
break;
|
||||
case COMPACT_VALIDATE:
|
||||
assert (args.length == 7);
|
||||
doCompactValidate(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6]);
|
||||
returnCode = 0;
|
||||
break;
|
||||
case COMPACT_REPAIR:
|
||||
assert (args.length == 8);
|
||||
doCompactRepair(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
|
||||
Boolean.valueOf(args[7]));
|
||||
returnCode = 0;
|
||||
break;
|
||||
case COMPACT_UNSCHEDULE_FILE:
|
||||
assert (args.length == 9);
|
||||
doCompactUnscheduleFile(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
|
||||
Boolean.valueOf(args[7]), Boolean.valueOf(args[8]));
|
||||
returnCode = 0;
|
||||
break;
|
||||
case COMPACT_UNSCHEDULE_PLAN:
|
||||
assert (args.length == 9);
|
||||
doCompactUnschedule(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
|
||||
Boolean.valueOf(args[7]), Boolean.valueOf(args[8]));
|
||||
returnCode = 0;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
System.exit(returnCode);
|
||||
}
|
||||
|
||||
private static int dataLoad(JavaSparkContext jsc, String command,
|
||||
String srcPath, String targetPath, String tableName,
|
||||
String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile, String sparkMaster,
|
||||
String sparkMemory, int retry) throws Exception {
|
||||
Config cfg = new Config();
|
||||
cfg.command = command;
|
||||
cfg.srcPath = srcPath;
|
||||
cfg.targetPath = targetPath;
|
||||
cfg.tableName = tableName;
|
||||
cfg.tableType = tableType;
|
||||
cfg.rowKey = rowKey;
|
||||
cfg.partitionKey = partitionKey;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.schemaFile = schemaFile;
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
return new HDFSParquetImporter(cfg).dataImport(jsc, retry);
|
||||
}
|
||||
|
||||
private static void doCompactValidate(JavaSparkContext jsc, String basePath, String compactionInstant,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory) throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.VALIDATE;
|
||||
cfg.outputPath = outputPath;
|
||||
cfg.compactionInstantTime = compactionInstant;
|
||||
cfg.parallelism = parallelism;
|
||||
if ((null != sparkMaster) && (!sparkMaster.isEmpty())) {
|
||||
jsc.getConf().setMaster(sparkMaster);
|
||||
}
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static void doCompactRepair(JavaSparkContext jsc, String basePath, String compactionInstant,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean dryRun) throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.REPAIR;
|
||||
cfg.outputPath = outputPath;
|
||||
cfg.compactionInstantTime = compactionInstant;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.dryRun = dryRun;
|
||||
if ((null != sparkMaster) && (!sparkMaster.isEmpty())) {
|
||||
jsc.getConf().setMaster(sparkMaster);
|
||||
}
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static void doCompactUnschedule(JavaSparkContext jsc, String basePath, String compactionInstant,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean skipValidation,
|
||||
boolean dryRun) throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.UNSCHEDULE_PLAN;
|
||||
cfg.outputPath = outputPath;
|
||||
cfg.compactionInstantTime = compactionInstant;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.dryRun = dryRun;
|
||||
cfg.skipValidation = skipValidation;
|
||||
if ((null != sparkMaster) && (!sparkMaster.isEmpty())) {
|
||||
jsc.getConf().setMaster(sparkMaster);
|
||||
}
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static void doCompactUnscheduleFile(JavaSparkContext jsc, String basePath, String fileId,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean skipValidation,
|
||||
boolean dryRun) throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.UNSCHEDULE_FILE;
|
||||
cfg.outputPath = outputPath;
|
||||
cfg.fileId = fileId;
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.dryRun = dryRun;
|
||||
cfg.skipValidation = skipValidation;
|
||||
if ((null != sparkMaster) && (!sparkMaster.isEmpty())) {
|
||||
jsc.getConf().setMaster(sparkMaster);
|
||||
}
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static int compact(JavaSparkContext jsc, String basePath, String tableName, String compactionInstant,
|
||||
int parallelism, String schemaFile, String sparkMemory, int retry, boolean schedule) throws Exception {
|
||||
HoodieCompactor.Config cfg = new HoodieCompactor.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.tableName = tableName;
|
||||
cfg.compactionInstantTime = compactionInstant;
|
||||
// TODO: Make this configurable along with strategy specific config - For now, this is a generic enough strategy
|
||||
cfg.strategyClassName = UnBoundedCompactionStrategy.class.getCanonicalName();
|
||||
cfg.parallelism = parallelism;
|
||||
cfg.schemaFile = schemaFile;
|
||||
cfg.runSchedule = schedule;
|
||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||
return new HoodieCompactor(cfg).compact(jsc, retry);
|
||||
}
|
||||
|
||||
private static int deduplicatePartitionPath(JavaSparkContext jsc, String duplicatedPartitionPath,
|
||||
String repairedOutputPath, String basePath) throws Exception {
|
||||
DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc),
|
||||
FSUtils.getFs(basePath, jsc.hadoopConfiguration()));
|
||||
job.fixDuplicates(true);
|
||||
return 0;
|
||||
}
|
||||
|
||||
private static int rollback(JavaSparkContext jsc, String commitTime, String basePath) throws Exception {
|
||||
HoodieWriteClient client = createHoodieClient(jsc, basePath);
|
||||
if (client.rollback(commitTime)) {
|
||||
LOG.info(String.format("The commit \"%s\" rolled back.", commitTime));
|
||||
return 0;
|
||||
} else {
|
||||
LOG.info(String.format("The commit \"%s\" failed to roll back.", commitTime));
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception {
|
||||
HoodieWriteClient client = createHoodieClient(jsc, basePath);
|
||||
if (client.rollbackToSavepoint(savepointTime)) {
|
||||
LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime));
|
||||
return 0;
|
||||
} else {
|
||||
LOG.info(String.format("The commit \"%s\" failed to roll back.", savepointTime));
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
return new HoodieWriteClient(jsc, config);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,171 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import com.codahale.metrics.Histogram;
|
||||
import com.codahale.metrics.Snapshot;
|
||||
import com.codahale.metrics.UniformReservoir;
|
||||
import java.io.IOException;
|
||||
import java.text.DecimalFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.NumericUtils;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class StatsCommand implements CommandMarker {
|
||||
|
||||
private static final int MAX_FILES = 1000000;
|
||||
|
||||
@CliAvailabilityIndicator({"stats wa"})
|
||||
public boolean isWriteAmpAvailable() {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many "
|
||||
+ "records were actually written")
|
||||
public String writeAmplificationStats(
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false")
|
||||
final boolean headerOnly) throws IOException {
|
||||
|
||||
long totalRecordsUpserted = 0;
|
||||
long totalRecordsWritten = 0;
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
int i = 0;
|
||||
DecimalFormat df = new DecimalFormat("#.00");
|
||||
for (HoodieInstant commitTime : timeline.getInstants().collect(Collectors.toList())) {
|
||||
String waf = "0";
|
||||
HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitTime).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
if (commit.fetchTotalUpdateRecordsWritten() > 0) {
|
||||
waf = df.format((float) commit.fetchTotalRecordsWritten() / commit.fetchTotalUpdateRecordsWritten());
|
||||
}
|
||||
rows.add(new Comparable[]{commitTime.getTimestamp(), commit.fetchTotalUpdateRecordsWritten(),
|
||||
commit.fetchTotalRecordsWritten(), waf});
|
||||
totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten();
|
||||
totalRecordsWritten += commit.fetchTotalRecordsWritten();
|
||||
}
|
||||
String waf = "0";
|
||||
if (totalRecordsUpserted > 0) {
|
||||
waf = df.format((float) totalRecordsWritten / totalRecordsUpserted);
|
||||
}
|
||||
rows.add(new Comparable[]{"Total", totalRecordsUpserted, totalRecordsWritten, waf});
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("CommitTime")
|
||||
.addTableHeaderField("Total Upserted")
|
||||
.addTableHeaderField("Total Written")
|
||||
.addTableHeaderField("Write Amplifiation Factor");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
private Comparable[] printFileSizeHistogram(String commitTime, Snapshot s) {
|
||||
return new Comparable[]{commitTime, s.getMin(),
|
||||
s.getValue(0.1), s.getMedian(),
|
||||
s.getMean(), s.get95thPercentile(),
|
||||
s.getMax(), s.size(),
|
||||
s.getStdDev()};
|
||||
}
|
||||
|
||||
@CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files")
|
||||
public String fileSizeStats(
|
||||
@CliOption(key = {"partitionPath"},
|
||||
help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") final String globRegex,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false")
|
||||
final boolean headerOnly) throws IOException {
|
||||
|
||||
FileSystem fs = HoodieCLI.fs;
|
||||
String globPath = String.format("%s/%s/*", HoodieCLI.tableMetadata.getBasePath(), globRegex);
|
||||
FileStatus[] statuses = fs.globStatus(new Path(globPath));
|
||||
|
||||
// max, min, #small files < 10MB, 50th, avg, 95th
|
||||
Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES));
|
||||
HashMap<String, Histogram> commitHistoMap = new HashMap<String, Histogram>();
|
||||
for (FileStatus fileStatus : statuses) {
|
||||
String commitTime = FSUtils.getCommitTime(fileStatus.getPath().getName());
|
||||
long sz = fileStatus.getLen();
|
||||
if (!commitHistoMap.containsKey(commitTime)) {
|
||||
commitHistoMap.put(commitTime, new Histogram(new UniformReservoir(MAX_FILES)));
|
||||
}
|
||||
commitHistoMap.get(commitTime).update(sz);
|
||||
globalHistogram.update(sz);
|
||||
}
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
int ind = 0;
|
||||
for (String commitTime : commitHistoMap.keySet()) {
|
||||
Snapshot s = commitHistoMap.get(commitTime).getSnapshot();
|
||||
rows.add(printFileSizeHistogram(commitTime, s));
|
||||
}
|
||||
Snapshot s = globalHistogram.getSnapshot();
|
||||
rows.add(printFileSizeHistogram("ALL", s));
|
||||
|
||||
Function<Object, String> converterFunction = entry ->
|
||||
NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Min", converterFunction);
|
||||
fieldNameToConverterMap.put("10th", converterFunction);
|
||||
fieldNameToConverterMap.put("50th", converterFunction);
|
||||
fieldNameToConverterMap.put("avg", converterFunction);
|
||||
fieldNameToConverterMap.put("95th", converterFunction);
|
||||
fieldNameToConverterMap.put("Max", converterFunction);
|
||||
fieldNameToConverterMap.put("StdDev", converterFunction);
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("CommitTime")
|
||||
.addTableHeaderField("Min")
|
||||
.addTableHeaderField("10th")
|
||||
.addTableHeaderField("50th")
|
||||
.addTableHeaderField("avg")
|
||||
.addTableHeaderField("95th")
|
||||
.addTableHeaderField("Max")
|
||||
.addTableHeaderField("NumFiles")
|
||||
.addTableHeaderField("StdDev");
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* 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.cli.commands;
|
||||
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
import org.springframework.stereotype.Component;
|
||||
|
||||
@Component
|
||||
public class UtilsCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "utils loadClass", help = "Load a class")
|
||||
public String loadClass(@CliOption(key = {"class"}, help = "Check mode") final String clazz) throws Exception {
|
||||
Class klass = Class.forName(clazz);
|
||||
return klass.getProtectionDomain().getCodeSource().getLocation().toExternalForm();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* 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.cli.utils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
|
||||
public class CommitUtil {
|
||||
|
||||
public static long countNewRecords(HoodieTableMetaClient target, List<String> commitsToCatchup) throws IOException {
|
||||
long totalNew = 0;
|
||||
HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants();
|
||||
for (String commit : commitsToCatchup) {
|
||||
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(
|
||||
timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten();
|
||||
}
|
||||
return totalNew;
|
||||
}
|
||||
}
|
||||
123
hudi-cli/src/main/java/org/apache/hudi/cli/utils/HiveUtil.java
Normal file
123
hudi-cli/src/main/java/org/apache/hudi/cli/utils/HiveUtil.java
Normal file
@@ -0,0 +1,123 @@
|
||||
/*
|
||||
* 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.cli.utils;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import javax.sql.DataSource;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
public class HiveUtil {
|
||||
|
||||
private static final String driverName = "org.apache.hive.jdbc.HiveDriver";
|
||||
|
||||
static {
|
||||
try {
|
||||
Class.forName(driverName);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
||||
}
|
||||
}
|
||||
|
||||
private static Connection connection;
|
||||
|
||||
private static Connection getConnection(String jdbcUrl, String user, String pass) throws SQLException {
|
||||
DataSource ds = getDatasource(jdbcUrl, user, pass);
|
||||
return ds.getConnection();
|
||||
}
|
||||
|
||||
private static DataSource getDatasource(String jdbcUrl, String user, String pass) {
|
||||
BasicDataSource ds = new BasicDataSource();
|
||||
ds.setDriverClassName(driverName);
|
||||
ds.setUrl(jdbcUrl);
|
||||
ds.setUsername(user);
|
||||
ds.setPassword(pass);
|
||||
return ds;
|
||||
}
|
||||
|
||||
public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, String user, String pass)
|
||||
throws SQLException {
|
||||
Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass);
|
||||
ResultSet rs = null;
|
||||
Statement stmt = conn.createStatement();
|
||||
try {
|
||||
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat");
|
||||
stmt.execute("set hive.stats.autogather=false");
|
||||
rs = stmt.executeQuery(
|
||||
"select count(`_hoodie_commit_time`) as cnt from " + dbName + "."
|
||||
+ source.getTableConfig().getTableName());
|
||||
long count = -1;
|
||||
if (rs.next()) {
|
||||
count = rs.getLong("cnt");
|
||||
}
|
||||
System.out.println("Total records in " + source.getTableConfig().getTableName() + " is " + count);
|
||||
return count;
|
||||
} finally {
|
||||
if (rs != null) {
|
||||
rs.close();
|
||||
}
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, int partitions,
|
||||
String user, String pass) throws SQLException {
|
||||
DateTime dateTime = DateTime.now();
|
||||
String endDateStr = dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-"
|
||||
+ String.format("%02d", dateTime.getDayOfMonth());
|
||||
dateTime = dateTime.minusDays(partitions);
|
||||
String startDateStr = dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-"
|
||||
+ String.format("%02d", dateTime.getDayOfMonth());
|
||||
System.out.println("Start date " + startDateStr + " and end date " + endDateStr);
|
||||
return countRecords(jdbcUrl, source, srcDb, startDateStr, endDateStr, user, pass);
|
||||
}
|
||||
|
||||
private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, String startDateStr,
|
||||
String endDateStr, String user, String pass) throws SQLException {
|
||||
Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass);
|
||||
ResultSet rs = null;
|
||||
Statement stmt = conn.createStatement();
|
||||
try {
|
||||
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat");
|
||||
stmt.execute("set hive.stats.autogather=false");
|
||||
rs = stmt.executeQuery(
|
||||
"select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source.getTableConfig().getTableName()
|
||||
+ " where datestr>'" + startDateStr + "' and datestr<='" + endDateStr + "'");
|
||||
if (rs.next()) {
|
||||
return rs.getLong("cnt");
|
||||
}
|
||||
return -1;
|
||||
} finally {
|
||||
if (rs != null) {
|
||||
rs.close();
|
||||
}
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,61 @@
|
||||
/*
|
||||
* 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.cli.utils;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.logging.Logger;
|
||||
|
||||
public class InputStreamConsumer extends Thread {
|
||||
|
||||
protected static final Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName());
|
||||
private InputStream is;
|
||||
|
||||
public InputStreamConsumer(InputStream is) {
|
||||
this.is = is;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
InputStreamReader isr = new InputStreamReader(is);
|
||||
BufferedReader br = new BufferedReader(isr);
|
||||
String line;
|
||||
while ((line = br.readLine()) != null) {
|
||||
LOG.info(line);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
LOG.severe(ioe.toString());
|
||||
ioe.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
public static void captureOutput(Process p) {
|
||||
InputStreamConsumer stdout;
|
||||
InputStreamConsumer errout;
|
||||
errout = new InputStreamConsumer(p.getErrorStream());
|
||||
stdout = new InputStreamConsumer(p.getInputStream());
|
||||
errout.start();
|
||||
stdout.start();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -0,0 +1,80 @@
|
||||
/*
|
||||
* 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.cli.utils;
|
||||
|
||||
import java.io.File;
|
||||
import java.net.URISyntaxException;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.cli.commands.SparkMain;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
|
||||
public class SparkUtil {
|
||||
|
||||
public static Logger logger = Logger.getLogger(SparkUtil.class);
|
||||
public static final String DEFUALT_SPARK_MASTER = "yarn-client";
|
||||
|
||||
/**
|
||||
* TODO: Need to fix a bunch of hardcoded stuff here eg: history server, spark distro
|
||||
*/
|
||||
public static SparkLauncher initLauncher(String propertiesFile) throws URISyntaxException {
|
||||
String currentJar = new File(SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath())
|
||||
.getAbsolutePath();
|
||||
SparkLauncher sparkLauncher = new SparkLauncher().setAppResource(currentJar)
|
||||
.setMainClass(SparkMain.class.getName());
|
||||
|
||||
if (StringUtils.isNotEmpty(propertiesFile)) {
|
||||
sparkLauncher.setPropertiesFile(propertiesFile);
|
||||
}
|
||||
|
||||
File libDirectory = new File(new File(currentJar).getParent(), "lib");
|
||||
for (String library : libDirectory.list()) {
|
||||
sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath());
|
||||
}
|
||||
return sparkLauncher;
|
||||
}
|
||||
|
||||
public static JavaSparkContext initJavaSparkConf(String name) {
|
||||
SparkConf sparkConf = new SparkConf().setAppName(name);
|
||||
String defMasterFromEnv = sparkConf.get("spark.master");
|
||||
if ((null == defMasterFromEnv) || (defMasterFromEnv.isEmpty())) {
|
||||
sparkConf.setMaster(DEFUALT_SPARK_MASTER);
|
||||
}
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||
sparkConf.set("spark.eventLog.overwrite", "true");
|
||||
sparkConf.set("spark.eventLog.enabled", "true");
|
||||
|
||||
// Configure hadoop conf
|
||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||
|
||||
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||
jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false);
|
||||
FSUtils.prepareHadoopConf(jsc.hadoopConfiguration());
|
||||
return jsc;
|
||||
}
|
||||
}
|
||||
614
hudi-cli/src/main/resources/META-INF/LICENSE.txt
Normal file
614
hudi-cli/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/.
|
||||
246
hudi-cli/src/main/resources/META-INF/NOTICE.txt
Normal file
246
hudi-cli/src/main/resources/META-INF/NOTICE.txt
Normal file
@@ -0,0 +1,246 @@
|
||||
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:
|
||||
An open source Java toolkit for Amazon S3 under Apache License, Version 2.0
|
||||
Antlr 3 Runtime under BSD
|
||||
ANTLR 4 Runtime under The BSD License
|
||||
ANTLR StringTemplate 4.0.2 under BSD licence
|
||||
AOP alliance under Public Domain
|
||||
aopalliance version 1.0 repackaged as a module under CDDL + GPLv2 with classpath exception
|
||||
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 Commons Collections under Apache License, Version 2.0
|
||||
Apache Commons Configuration under Apache License, Version 2.0
|
||||
Apache Commons Crypto under Apache License, Version 2.0
|
||||
Apache Commons IO under Apache License, Version 2.0
|
||||
Apache Commons Lang under Apache License, Version 2.0
|
||||
Apache Commons Logging under The Apache Software License, Version 2.0
|
||||
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 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 - Common 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 Kafka under The Apache Software License, Version 2.0
|
||||
Apache Log4j 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 (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 Velocity under The Apache Software License, Version 2.0
|
||||
Apache XBean :: ASM 5 shaded (repackaged) under null or null
|
||||
ApacheDS I18n under The Apache Software License, Version 2.0
|
||||
ApacheDS Protocol Kerberos Codec under The Apache Software License, Version 2.0
|
||||
ASCII List under Apache 2
|
||||
Ascii Table under Apache 2
|
||||
ASM Core under 3-Clause BSD License
|
||||
Bean Validation API under The Apache Software License, Version 2.0
|
||||
bijection-avro under Apache 2
|
||||
bijection-core under Apache 2
|
||||
chill under Apache 2
|
||||
chill-java under Apache 2
|
||||
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
|
||||
Compress-LZF under Apache License 2.0
|
||||
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
|
||||
Digester under The Apache Software License, Version 2.0
|
||||
empty under The Apache 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
|
||||
Flip Tables under Apache 2.0
|
||||
Fluent API for Apache HttpClient under Apache License, Version 2.0
|
||||
Graphite Integration for Metrics under Apache License 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-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-server-common under Apache License, Version 2.0
|
||||
Hamcrest Core under BSD style
|
||||
HK2 API module under CDDL + GPLv2 with classpath exception
|
||||
HK2 Implementation Utilities under CDDL + GPLv2 with classpath exception
|
||||
hoodie-cli under Apache License, Version 2.0
|
||||
hoodie-client under Apache License, Version 2.0
|
||||
hoodie-common under Apache License, Version 2.0
|
||||
hoodie-hadoop-mr under Apache License, Version 2.0
|
||||
hoodie-hive under Apache License, Version 2.0
|
||||
hoodie-spark under Apache License, Version 2.0
|
||||
hoodie-timeline-service under Apache License, Version 2.0
|
||||
hoodie-utilities under Apache License, Version 2.0
|
||||
htrace-core under The Apache Software License, Version 2.0
|
||||
HttpClient under Apache License
|
||||
IntelliJ IDEA Annotations under The Apache Software License, Version 2.0
|
||||
io.confluent:common-config under Apache License, Version 2.0
|
||||
io.confluent:common-utils under Apache License, Version 2.0
|
||||
io.confluent:kafka-avro-serializer under Apache License, Version 2.0
|
||||
io.confluent:kafka-schema-registry-client under Apache License, Version 2.0
|
||||
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
|
||||
Jackson-module-paranamer under The Apache Software License, Version 2.0
|
||||
jackson-module-scala under The Apache Software License, Version 2.0
|
||||
Janino under New BSD License
|
||||
java-xmlbuilder under Apache License, Version 2.0
|
||||
Javalin under The Apache Software License, Version 2.0
|
||||
Javassist under MPL 1.1 or LGPL 2.1 or Apache License 2.0
|
||||
javax.annotation API under CDDL + GPLv2 with classpath exception
|
||||
javax.inject:1 as OSGi bundle under CDDL + GPLv2 with classpath exception
|
||||
javax.ws.rs-api under CDDL 1.1 or GPL2 w/ CPE
|
||||
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
|
||||
JCL 1.1.1 implemented over SLF4J under MIT License
|
||||
JCodings under MIT License
|
||||
jcommander under Apache 2.0
|
||||
jersey-client under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-container-servlet under CDDL+GPL License
|
||||
jersey-container-servlet-core under CDDL+GPL License
|
||||
jersey-core under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-core-client under CDDL+GPL License
|
||||
jersey-core-common under CDDL+GPL License
|
||||
jersey-core-server under CDDL+GPL License
|
||||
jersey-json under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-media-jaxb under CDDL+GPL License
|
||||
jersey-repackaged-guava under CDDL+GPL License
|
||||
jersey-server under CDDL 1.1 or GPL2 w/ CPE
|
||||
Jettison under Apache License, Version 2.0
|
||||
Jetty :: Asynchronous HTTP Client under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Continuation under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Http Utility under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: IO Utility under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Security under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Server Core under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Servlet Handling under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Webapp Application Support under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: API under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Client under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Common under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Server under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Servlet Interface under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: XML utilities 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 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
|
||||
JSch under BSD
|
||||
json4s-ast under ASL
|
||||
json4s-core under ASL
|
||||
json4s-jackson under ASL
|
||||
jsp-api under CDDL
|
||||
JUL to SLF4J bridge under MIT License
|
||||
JUnit under Common Public License Version 1.0
|
||||
JVM Integration for Metrics under Apache License 2.0
|
||||
Kryo Shaded under 3-Clause BSD License
|
||||
leveldbjni-all under The BSD 3-Clause License
|
||||
LZ4 and xxHash under The Apache Software License, Version 2.0
|
||||
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
|
||||
org.jetbrains.kotlin:kotlin-stdlib under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-common under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-jdk7 under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-jdk8 under The Apache License, Version 2.0
|
||||
oro under Apache License, Version 2.0
|
||||
OSGi resource locator bundle - used by various API providers that rely on META-INF/services mechanism to locate providers. under CDDL + GPLv2 with classpath exception
|
||||
ParaNamer Core under BSD
|
||||
Protocol Buffer Java API under New BSD license
|
||||
Py4J under The New BSD License
|
||||
pyrolite under MIT License
|
||||
RabbitMQ Java Client under ASL 2.0 or GPL v2 or MPL 1.1
|
||||
RoaringBitmap under Apache 2
|
||||
RocksDB JNI under Apache License 2.0 or GNU General Public License, version 2
|
||||
Scala Compiler under BSD 3-Clause
|
||||
Scala Library under BSD 3-Clause
|
||||
scala-parser-combinators under BSD 3-clause
|
||||
scala-xml under BSD 3-clause
|
||||
Scalap under BSD 3-Clause
|
||||
scalatest under the Apache License, ASL Version 2.0
|
||||
ServiceLocator Default Implementation under CDDL + GPLv2 with classpath exception
|
||||
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
|
||||
snappy-java under The Apache Software License, Version 2.0
|
||||
Spark Integration for Kafka 0.8 under Apache 2.0 License
|
||||
Spark Project Catalyst under Apache 2.0 License
|
||||
Spark Project Core under Apache 2.0 License
|
||||
Spark Project Launcher under Apache 2.0 License
|
||||
Spark Project Networking under Apache 2.0 License
|
||||
Spark Project Shuffle Streaming Service under Apache 2.0 License
|
||||
Spark Project Sketch under Apache 2.0 License
|
||||
Spark Project SQL under Apache 2.0 License
|
||||
Spark Project Tags under Apache 2.0 License
|
||||
Spark Project Unsafe under Apache 2.0 License
|
||||
spark-avro under Apache-2.0
|
||||
Spring AOP under The Apache Software License, Version 2.0
|
||||
Spring Beans under The Apache Software License, Version 2.0
|
||||
Spring Context under The Apache Software License, Version 2.0
|
||||
Spring Context Support under The Apache Software License, Version 2.0
|
||||
Spring Core under The Apache Software License, Version 2.0
|
||||
Spring Expression Language (SpEL) under The Apache Software License, Version 2.0
|
||||
Spring Shell under The Apache Software License, Version 2.0
|
||||
stream-lib under Apache License, Version 2.0
|
||||
The Netty Project under Apache License, Version 2.0
|
||||
univocity-parsers under Apache 2
|
||||
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
|
||||
ZkClient under The Apache Software License, Version 2.0
|
||||
zookeeper under Apache License, Version 2.0
|
||||
|
||||
@@ -0,0 +1,28 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<beans xmlns="http://www.springframework.org/schema/beans"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xmlns:context="http://www.springframework.org/schema/context"
|
||||
xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
|
||||
http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context.xsd">
|
||||
|
||||
<context:component-scan base-package="org.apache.hudi.cli"/>
|
||||
|
||||
</beans>
|
||||
193
hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala
Normal file
193
hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala
Normal file
@@ -0,0 +1,193 @@
|
||||
/*
|
||||
* 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.cli
|
||||
|
||||
import java.util.stream.Collectors
|
||||
|
||||
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
|
||||
import org.apache.hudi.common.model.{HoodieDataFile, HoodieRecord}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.util.FSUtils
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.log4j.Logger
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable._
|
||||
|
||||
|
||||
/**
|
||||
* Spark job to de-duplicate data present in a partition path
|
||||
*/
|
||||
class DedupeSparkJob(basePath: String,
|
||||
duplicatedPartitionPath: String,
|
||||
repairOutputPath: String,
|
||||
sqlContext: SQLContext,
|
||||
fs: FileSystem) {
|
||||
|
||||
|
||||
val sparkHelper = new SparkHelper(sqlContext, fs)
|
||||
val LOG = Logger.getLogger(this.getClass)
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* @param tblName
|
||||
* @return
|
||||
*/
|
||||
def getDupeKeyDF(tblName: String): DataFrame = {
|
||||
val dupeSql =
|
||||
s"""
|
||||
select `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` as dupe_key,
|
||||
count(*) as dupe_cnt
|
||||
from ${tblName}
|
||||
group by `${HoodieRecord.RECORD_KEY_METADATA_FIELD}`
|
||||
having dupe_cnt > 1
|
||||
"""
|
||||
return sqlContext.sql(dupeSql)
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* Check a given partition for duplicates and suggest the deletions that need to be done in each file,
|
||||
* in order to set things right.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
private def planDuplicateFix(): HashMap[String, HashSet[String]] = {
|
||||
|
||||
val tmpTableName = s"htbl_${System.currentTimeMillis()}"
|
||||
val dedupeTblName = s"${tmpTableName}_dupeKeys"
|
||||
|
||||
val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
|
||||
|
||||
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"$basePath/$duplicatedPartitionPath"))
|
||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||
val filteredStatuses = latestFiles.map(f => f.getPath)
|
||||
LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}")
|
||||
|
||||
val df = sqlContext.parquetFile(filteredStatuses: _*)
|
||||
df.registerTempTable(tmpTableName)
|
||||
val dupeKeyDF = getDupeKeyDF(tmpTableName)
|
||||
dupeKeyDF.registerTempTable(dedupeTblName)
|
||||
|
||||
// Obtain necessary satellite information for duplicate rows
|
||||
val dupeDataSql =
|
||||
s"""
|
||||
SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time`
|
||||
FROM $tmpTableName h
|
||||
JOIN $dedupeTblName d
|
||||
ON h.`_hoodie_record_key` = d.dupe_key
|
||||
"""
|
||||
val dupeMap = sqlContext.sql(dupeDataSql).collectAsList().groupBy(r => r.getString(0))
|
||||
val fileToDeleteKeyMap = new HashMap[String, HashSet[String]]()
|
||||
|
||||
// Mark all files except the one with latest commits for deletion
|
||||
dupeMap.foreach(rt => {
|
||||
val (key, rows) = rt
|
||||
var maxCommit = -1L
|
||||
|
||||
rows.foreach(r => {
|
||||
val c = r(3).asInstanceOf[String].toLong
|
||||
if (c > maxCommit)
|
||||
maxCommit = c
|
||||
})
|
||||
|
||||
rows.foreach(r => {
|
||||
val c = r(3).asInstanceOf[String].toLong
|
||||
if (c != maxCommit) {
|
||||
val f = r(2).asInstanceOf[String].split("_")(0)
|
||||
if (!fileToDeleteKeyMap.contains(f)) {
|
||||
fileToDeleteKeyMap(f) = HashSet[String]()
|
||||
}
|
||||
fileToDeleteKeyMap(f).add(key)
|
||||
}
|
||||
})
|
||||
})
|
||||
return fileToDeleteKeyMap
|
||||
}
|
||||
|
||||
|
||||
def fixDuplicates(dryRun: Boolean = true) = {
|
||||
val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
|
||||
|
||||
val allFiles = fs.listStatus(new Path(s"$basePath/$duplicatedPartitionPath"))
|
||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||
|
||||
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||
|
||||
val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap
|
||||
val dupeFixPlan = planDuplicateFix()
|
||||
|
||||
// 1. Copy all latest files into the temp fix path
|
||||
fileNameToPathMap.foreach { case (fileName, filePath) =>
|
||||
val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else ""
|
||||
val dstPath = new Path(s"$repairOutputPath/${filePath.getName}$badSuffix")
|
||||
LOG.info(s"Copying from $filePath to $dstPath")
|
||||
FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf)
|
||||
}
|
||||
|
||||
// 2. Remove duplicates from the bad files
|
||||
dupeFixPlan.foreach { case (fileName, keysToSkip) =>
|
||||
val commitTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName)
|
||||
val badFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}.bad")
|
||||
val newFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}")
|
||||
LOG.info(" Skipping and writing new file for : " + fileName)
|
||||
SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName))
|
||||
fs.delete(badFilePath, false)
|
||||
}
|
||||
|
||||
// 3. Check that there are no duplicates anymore.
|
||||
val df = sqlContext.read.parquet(s"$repairOutputPath/*.parquet")
|
||||
df.registerTempTable("fixedTbl")
|
||||
val dupeKeyDF = getDupeKeyDF("fixedTbl")
|
||||
val dupeCnt = dupeKeyDF.count()
|
||||
if (dupeCnt != 0) {
|
||||
dupeKeyDF.show()
|
||||
throw new HoodieException("Still found some duplicates!!.. Inspect output")
|
||||
}
|
||||
|
||||
// 4. Additionally ensure no record keys are left behind.
|
||||
val sourceDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => t._2.toString).toList)
|
||||
val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"$repairOutputPath/${t._2.getName}").toList)
|
||||
val missedRecordKeysDF = sourceDF.except(fixedDF)
|
||||
val missedCnt = missedRecordKeysDF.count()
|
||||
if (missedCnt != 0) {
|
||||
missedRecordKeysDF.show()
|
||||
throw new HoodieException("Some records in source are not found in fixed files. Inspect output!!")
|
||||
}
|
||||
|
||||
|
||||
println("No duplicates found & counts are in check!!!! ")
|
||||
// 4. Prepare to copy the fixed files back.
|
||||
fileNameToPathMap.foreach { case (_, filePath) =>
|
||||
val srcPath = new Path(s"$repairOutputPath/${filePath.getName}")
|
||||
val dstPath = new Path(s"$basePath/$duplicatedPartitionPath/${filePath.getName}")
|
||||
if (dryRun) {
|
||||
LOG.info(s"[JUST KIDDING!!!] Copying from $srcPath to $dstPath")
|
||||
} else {
|
||||
// for real
|
||||
LOG.info(s"[FOR REAL!!!] Copying from $srcPath to $dstPath")
|
||||
FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
138
hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala
Normal file
138
hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala
Normal file
@@ -0,0 +1,138 @@
|
||||
/*
|
||||
* 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.cli
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.IndexedRecord
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.util.ParquetUtils
|
||||
import org.apache.hudi.common.{BloomFilter, HoodieJsonPayload}
|
||||
import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
|
||||
import org.apache.hudi.io.storage.{HoodieParquetConfig, HoodieParquetWriter}
|
||||
import org.apache.parquet.avro.AvroSchemaConverter
|
||||
import org.apache.parquet.hadoop.ParquetFileReader
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable._
|
||||
|
||||
|
||||
object SparkHelpers {
|
||||
@throws[Exception]
|
||||
def skipKeysAndWriteNewFile(commitTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) {
|
||||
val sourceRecords = ParquetUtils.readAvroRecords(fs.getConf, sourceFile)
|
||||
val schema: Schema = sourceRecords.get(0).getSchema
|
||||
val filter: BloomFilter = new BloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble)
|
||||
val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter)
|
||||
val parquetConfig: HoodieParquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf, HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO.toDouble)
|
||||
val writer = new HoodieParquetWriter[HoodieJsonPayload, IndexedRecord](commitTime, destinationFile, parquetConfig, schema)
|
||||
for (rec <- sourceRecords) {
|
||||
val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString
|
||||
if (!keysToSkip.contains(key)) {
|
||||
|
||||
writer.writeAvro(key, rec)
|
||||
}
|
||||
}
|
||||
writer.close
|
||||
}
|
||||
|
||||
def getBloomFilter(file: String, conf: Configuration): String = {
|
||||
val footer = ParquetFileReader.readFooter(conf, new Path(file));
|
||||
return footer.getFileMetaData().getKeyValueMetaData().get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Bunch of Spark Shell/Scala stuff useful for debugging
|
||||
*/
|
||||
class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
||||
|
||||
|
||||
/**
|
||||
* Print keys from a file
|
||||
*
|
||||
* @param file
|
||||
*/
|
||||
def printKeysFromFile(file: String) = {
|
||||
getRowKeyDF(file).collect().foreach(println(_))
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param file
|
||||
* @return
|
||||
*/
|
||||
def getRowKeyDF(file: String): DataFrame = {
|
||||
sqlContext.read.parquet(file).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Does the rowKey actually exist in the file.
|
||||
*
|
||||
* @param rowKey
|
||||
* @param file
|
||||
* @return
|
||||
*/
|
||||
def isFileContainsKey(rowKey: String, file: String): Boolean = {
|
||||
println(s"Checking $file for key $rowKey")
|
||||
val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '$rowKey'")
|
||||
if (ff.count() > 0) true else false
|
||||
}
|
||||
|
||||
/**
|
||||
* Number of keys in a given file
|
||||
*
|
||||
* @param file
|
||||
* @param sqlContext
|
||||
*/
|
||||
def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) = {
|
||||
println(getRowKeyDF(file).collect().length)
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* Checks that all the keys in the file, have been added to the bloom filter
|
||||
* in the footer
|
||||
*
|
||||
* @param conf
|
||||
* @param sqlContext
|
||||
* @param file
|
||||
* @return
|
||||
*/
|
||||
def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String): Boolean = {
|
||||
val bfStr = SparkHelpers.getBloomFilter(file, conf)
|
||||
val bf = new BloomFilter(bfStr)
|
||||
val foundCount = sqlContext.parquetFile(file)
|
||||
.select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
|
||||
.collect().count(r => !bf.mightContain(r.getString(0)))
|
||||
val totalCount = getKeyCount(file, sqlContext)
|
||||
s"totalCount: ${totalCount}, foundCount: ${foundCount}"
|
||||
totalCount == foundCount
|
||||
}
|
||||
|
||||
def getDistinctKeyDF(paths: List[String]): DataFrame = {
|
||||
sqlContext.read.parquet(paths: _*).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`").distinct()
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user