hudi-bot opened a new issue, #14596: URL: https://github.com/apache/hudi/issues/14596
If anyone wants to use custom upsert logic then they have to override the Latest avro payload class which is only possible in java or scala . Python developers have no such option . Will be introducing a new payload class and a new key which can work in java , scala and python This class will be responsible for custom upsert logic and a new key hoodie.update.key which will accept the columns which only need to be updated "hoodie.update.keys": "admission_date,name", #comma seperated key "hoodie.datasource.write.payload.class": "com.hudiUpsert.hudiCustomUpsert" #custom upsert key so this will only update the column admission_date and name in the target table ## JIRA info - Link: https://issues.apache.org/jira/browse/HUDI-1936 - Type: New Feature --- ## Comments 25/Jul/21 14:50;githubbot;fanaticjo commented on a change in pull request #3035: URL: https://github.com/apache/hudi/pull/3035#discussion_r676157680 ########## File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithCustomAvroPayload.java ########## @@ -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.common.model; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.ColumnNotFoundException; +import org.apache.hudi.exception.UpdateKeyNotFoundException; +import org.apache.hudi.exception.WriteOperationException; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; + +/** + * subclass of OverwriteWithLatestAvroPayload used for delta streamer. + * + * <ol> + * <li> combineAndGetUpdateValue - Accepts the column names to be updated; + * <li> splitKeys - Split keys based upon keys; + * </ol> + */ +public class OverwriteWithCustomAvroPayload extends OverwriteWithLatestAvroPayload { + + public OverwriteWithCustomAvroPayload(GenericRecord record, Comparable orderingVal) { + super(record, orderingVal); + } + + /** + * split keys over. + */ + public List<String> splitKeys(String keys) throws UpdateKeyNotFoundException { + if (keys == null) { + throw new UpdateKeyNotFoundException("keys cannot be null"); + } else if (keys.equals("")) { + throw new UpdateKeyNotFoundException("keys cannot be blank"); + } else { + return Arrays.stream(keys.split(",")).collect(Collectors.toList()); + } + } + + /** + * check column exi. + */ + public boolean checkColumnExists(List<String> keys, Schema schema) { + List<Schema.Field> field = schema.getFields(); + List<Schema.Field> common = new ArrayList<>(); + for (Schema.Field columns : field) { + if (keys.contains(columns.name())) { + common.add(columns); + } + } + return common.size() == keys.size(); + } + + @Override + public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) + throws WriteOperationException, IOException, ColumnNotFoundException, UpdateKeyNotFoundException { + + if (!properties.getProperty("hoodie.datasource.write.operation").equals("upsert")) { + throw new WriteOperationException("write should be upsert"); + } + + Option<IndexedRecord> recordOption = getInsertValue(schema); + + if (!recordOption.isPresent()) { + return Option.empty(); + } + Review comment: @nsivabalan any updates ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] ;;; --- 08/Aug/21 20:18;githubbot;hudi-bot edited a comment on pull request #3035: URL: https://github.com/apache/hudi/pull/3035#issuecomment-862017744 <!-- Meta data { "version" : 1, "metaDataEntries" : [ { "hash" : "26dadb6627c90c9f06e66fba0b8bd24e5579665f", "status" : "FAILURE", "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=209", "triggerID" : "26dadb6627c90c9f06e66fba0b8bd24e5579665f", "triggerType" : "PUSH" } ] }--> ## CI report: * 26dadb6627c90c9f06e66fba0b8bd24e5579665f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=209) <details> <summary>Bot commands</summary> The @flinkbot bot supports the following commands: - `@flinkbot run travis` re-run the last Travis build - `@flinkbot run azure` re-run the last Azure build </details> -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] ;;; --- 09/Aug/21 04:23;githubbot;hudi-bot edited a comment on pull request #3035: URL: https://github.com/apache/hudi/pull/3035#issuecomment-862017744 <!-- Meta data { "version" : 1, "metaDataEntries" : [ { "hash" : "26dadb6627c90c9f06e66fba0b8bd24e5579665f", "status" : "FAILURE", "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=209", "triggerID" : "26dadb6627c90c9f06e66fba0b8bd24e5579665f", "triggerType" : "PUSH" } ] }--> ## CI report: * 26dadb6627c90c9f06e66fba0b8bd24e5579665f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=209) <details> <summary>Bot commands</summary> @hudi-bot supports the following commands: - `@hudi-bot run travis` re-run the last Travis build - `@hudi-bot run azure` re-run the last Azure build </details> -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] ;;; --- 10/Aug/21 21:50;githubbot;vinothchandar commented on pull request #3035: URL: https://github.com/apache/hudi/pull/3035#issuecomment-896336864 In some sense, with the Spark SQL support now, python users can do custom merges? does that satisfy your requirements? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] ;;; -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
