dengzhhu653 commented on code in PR #6450: URL: https://github.com/apache/hive/pull/6450#discussion_r3331481304
########## standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metastore/impl/NotificationStoreImpl.java: ########## @@ -0,0 +1,428 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.metastore.metastore.impl; + +import com.google.common.collect.Lists; + +import javax.jdo.Query; +import javax.jdo.datastore.JDOConnection; + +import java.sql.Connection; +import java.sql.Statement; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.collections4.CollectionUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.DatabaseProduct; +import org.apache.hadoop.hive.metastore.directsql.MetaStoreDirectSql; +import org.apache.hadoop.hive.metastore.PersistenceManagerProvider; +import org.apache.hadoop.hive.metastore.RawStore; +import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.NotificationEventRequest; +import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; +import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest; +import org.apache.hadoop.hive.metastore.api.NotificationEventsCountResponse; +import org.apache.hadoop.hive.metastore.api.WriteEventInfo; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.metastore.RawStoreAware; +import org.apache.hadoop.hive.metastore.model.MNotificationLog; +import org.apache.hadoop.hive.metastore.model.MNotificationNextId; +import org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog; +import org.apache.hadoop.hive.metastore.metastore.iface.NotificationStore; +import org.apache.hadoop.hive.metastore.tools.SQLGenerator; +import org.apache.hadoop.hive.metastore.utils.RetryingExecutor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.hive.metastore.ObjectStore.appendSimpleCondition; +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; +import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier; + +public class NotificationStoreImpl extends RawStoreAware implements NotificationStore { + private static final Logger LOG = LoggerFactory.getLogger(NotificationStoreImpl.class); + private Configuration conf; + private SQLGenerator sqlGenerator; + private MetaStoreDirectSql directSql; + + @Override + public void setBaseStore(RawStore store) { + super.setBaseStore(store); + this.conf = baseStore.getConf(); + DatabaseProduct dbType = PersistenceManagerProvider.getDatabaseProduct(); + this.sqlGenerator = new SQLGenerator(dbType, conf); + } + + @Override + public NotificationEventResponse getNextNotification(NotificationEventRequest rqst) { + NotificationEventResponse result = new NotificationEventResponse(); + result.setEvents(new ArrayList<>()); + long lastEvent = rqst.getLastEvent(); + List<Object> parameterVals = new ArrayList<>(); + parameterVals.add(lastEvent); + // filterBuilder parameter is used for construction of conditional clause in the select query + StringBuilder filterBuilder = new StringBuilder("eventId > para" + parameterVals.size()); + // parameterBuilder parameter is used for specify what types of parameters will go into the filterBuilder + StringBuilder parameterBuilder = new StringBuilder("java.lang.Long para" + parameterVals.size()); + /* A fully constructed query would like: + -> filterBuilder: eventId > para0 && catalogName == para1 && dbName == para2 && (tableName == para3 + || tableName == para4) && eventType != para5 + -> parameterBuilder: java.lang.Long para0, java.lang.String para1, java.lang.String para2 + , java.lang.String para3, java.lang.String para4, java.lang.String para5 + */ + if (rqst.isSetCatName()) { + parameterVals.add(normalizeIdentifier(rqst.getCatName())); + parameterBuilder.append(", java.lang.String para" + parameterVals.size()); + filterBuilder.append(" && catalogName == para" + parameterVals.size()); + } + if (rqst.isSetDbName()) { + parameterVals.add(normalizeIdentifier(rqst.getDbName())); + parameterBuilder.append(", java.lang.String para" + parameterVals.size()); + filterBuilder.append(" && dbName == para" + parameterVals.size()); + } + if (rqst.isSetTableNames() && !rqst.getTableNames().isEmpty()) { + filterBuilder.append(" && ("); + for (String tableName : rqst.getTableNames()) { + parameterVals.add(normalizeIdentifier(tableName)); + parameterBuilder.append(", java.lang.String para" + parameterVals.size()); + filterBuilder.append("tableName == para" + parameterVals.size()+ " || "); + } + filterBuilder.setLength(filterBuilder.length() - 4); // remove the last " || " + filterBuilder.append(") "); + } + if (rqst.isSetEventTypeList()) { + filterBuilder.append(" && ("); + for (String eventType : rqst.getEventTypeList()) { + parameterVals.add(eventType); + parameterBuilder.append(", java.lang.String para" + parameterVals.size()); + filterBuilder.append("eventType == para" + parameterVals.size() + " || "); + } + filterBuilder.setLength(filterBuilder.length() - 4); // remove the last " || " + filterBuilder.append(") "); + } + if (rqst.isSetEventTypeSkipList()) { + for (String eventType : rqst.getEventTypeSkipList()) { + parameterVals.add(eventType); + parameterBuilder.append(", java.lang.String para" + parameterVals.size()); + filterBuilder.append(" && eventType != para" + parameterVals.size()); + } + } + Query query = pm.newQuery(MNotificationLog.class, filterBuilder.toString()); + query.declareParameters(parameterBuilder.toString()); + query.setOrdering("eventId ascending"); + int maxEventResponse = MetastoreConf.getIntVar(baseStore.getConf(), MetastoreConf.ConfVars.METASTORE_MAX_EVENT_RESPONSE); + int maxEvents = (rqst.getMaxEvents() < maxEventResponse && rqst.getMaxEvents() > 0) ? rqst.getMaxEvents() : maxEventResponse; + query.setRange(0, maxEvents); + Collection<MNotificationLog> events = + (Collection) query.executeWithArray(parameterVals.toArray(new Object[0])); + if (events == null) { + return result; + } + Iterator<MNotificationLog> i = events.iterator(); + while (i.hasNext()) { + result.addToEvents(translateDbToThrift(i.next())); + } + return result; + } + + private NotificationEvent translateDbToThrift(MNotificationLog dbEvent) { + NotificationEvent event = new NotificationEvent(); + event.setEventId(dbEvent.getEventId()); + event.setEventTime(dbEvent.getEventTime()); + event.setEventType(dbEvent.getEventType()); + event.setCatName(dbEvent.getCatalogName()); + event.setDbName(dbEvent.getDbName()); + event.setTableName(dbEvent.getTableName()); + event.setMessage((dbEvent.getMessage())); + event.setMessageFormat(dbEvent.getMessageFormat()); + return event; + } + + private void lockNotificationSequenceForUpdate() throws MetaException { + int maxRetries = + MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.NOTIFICATION_SEQUENCE_LOCK_MAX_RETRIES); + long sleepInterval = MetastoreConf.getTimeVar(conf, + MetastoreConf.ConfVars.NOTIFICATION_SEQUENCE_LOCK_RETRY_SLEEP_INTERVAL, TimeUnit.MILLISECONDS); + if (sqlGenerator.getDbProduct().isDERBY()) { Review Comment: DataNucleus(Jdo) doesn't provide a direct way for locking a row/table, and we know current database is Derby, so the statements sent are Derby compliant and safe to run. -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
