You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "henrib (via GitHub)" <gi...@apache.org> on 2023/05/22 15:21:07 UTC

[GitHub] [hive] henrib opened a new pull request, #4194: HIVE-27186: A persistent property store

henrib opened a new pull request, #4194:
URL: https://github.com/apache/hive/pull/4194

   [https://issues.apache.org/jira/browse/HIVE-27186](https://issues.apache.org/jira/browse/HIVE-27186)
   A persistent property store usable as a support facility for any metadata augmentation feature.
   
   ### What changes were proposed in this pull request?
   A property-value model is the simple and generic exposed API.
   To provision for several usage scenarios, the model entry point is a 'namespace' that qualifies the feature-component property manager. For example, 'stats' could be the namespace for all properties related to the 'statistics' feature.
   The namespace identifies a manager that handles property-groups persisted as property-maps. For instance, all statistics pertaining to a given table would be collocated in the same property-group. As such, all properties (say number of 'unique_values' per columns) for a given HMS table 'relation0' would all be stored and persisted in the same property-map instance.
   Property-maps may be decorated by an (optional) schema that may declare the name and value-type of allowed properties (and their optional default value). Each property is addressed by a name, a path uniquely identifying the property in a given property map.
   The manager also handles transforming property-map names to the property-map keys used to persist them in the DB.
   
   The API provides inserting/updating properties in bulk transactionally. It also provides selection/projection to help reduce the volume of exchange between client/server; selection can use (JEXL expression) predicates to filter maps.
   
   
   ### Why are the changes needed?
   When adding new meta-data oriented features, we usually need to persist information linking the feature data and the HiveMetaStore objects it applies to. Any information related to a database, a table or the cluster - like statistics for example or any operational data state or data (think rolling backup) -  fall in this use-case.
   Typically, accommodating such a feature requires modifying the Metastore database schema by adding or altering a table. It also usually implies modifying the thrift APIs to expose such meta-data to consumers.
   The proposed feature wants to solve the persistence and query/transport for these types of use-cases by exposing a 'key/(meta)value' store exposed as a property system.
   
   ### Does this PR introduce _any_ user-facing change?
   It introduces new API calls.
   
   
   ### How was this patch tested?
   Junit + coverage


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1570717114

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [85 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1203897076


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -0,0 +1,268 @@
+/* * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonSyntaxException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyManager;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.Source;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * The property  cli servlet.
+ */
+public class PropertyServlet extends HttpServlet {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyServlet.class);
+  /** The object store. */
+  private final RawStore objectStore;

Review Comment:
   I'm afraid that the `ObjectStore` is not thread-safe, especially for the method `openTransaction` and `commitTransaction`



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1204597110


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Digester.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.properties;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Date;
+import java.util.UUID;
+
+/**
+ * Helper class that creates a type 5 uuid.
+ * <p>This is computed from a set of updates using a SHA-1 message digest massaged into an UUID.
+ * see <a href="https://en.wikipedia.org/wiki/Universally_unique_identifier">...</a>
+ */
+public class Digester {
+    /** The Namespace uuid. */
+    private final UUID nsuid;
+    /** The digest used to compute the UUID. */
+    private final MessageDigest md;
+    /** A default namespace based on the class loading time. */
+    private static final UUID TEMP_NS;
+    static {
+        MessageDigest md = createDigest();
+        digest(md, System.currentTimeMillis());
+        TEMP_NS = computeUUID(md);
+    }
+
+    /**
+     * Allows to update the message digest from an object.
+     */
+    private static class TagOutputStream extends OutputStream {
+        /** The digest to update. */
+        private final MessageDigest md;
+
+        /**
+         * Sole ctor.
+         * @param md the message digester
+         */
+        TagOutputStream(MessageDigest md) {
+            this.md = md;
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            md.update((byte) b);
+        }
+
+        @Override
+        public void write(byte[] b, int off, int len) throws IOException {
+            md.update(b, off, len);
+        }
+    }
+
+    /**
+     * @return a SHA-1 message digest
+     */
+    private static MessageDigest createDigest() {
+        MessageDigest md;
+        try {
+            md = MessageDigest.getInstance("SHA-1");
+        } catch (NoSuchAlgorithmException nsae) {
+            throw new InternalError("SHA not supported");
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with a uuid.
+     * @param md the digest to update
+     * @param uid the uuid
+     */
+    private static MessageDigest digest(MessageDigest md, UUID uid) {
+        if (uid != null) {
+            long msb = uid.getMostSignificantBits();
+            digest(md, msb);
+            long lsb = uid.getLeastSignificantBits();
+            digest(md, lsb);
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with an input stream.
+     * @param md the digest to update
+     * @param input the input to consume
+     * @throws IllegalStateException if an io exception occurs
+     */
+    private static void digest(MessageDigest md, InputStream input) {
+        try (OutputStream out = new TagOutputStream(md)) {
+            byte[] buffer = new byte[1024];
+            int read;
+            while ((read = input.read(buffer, 0, 1024)) >= 0) {
+                out.write(buffer, 0, read);
+            }
+        } catch(IOException xio) {
+            throw new IllegalStateException(xio);
+        }
+    }
+
+
+    /**
+     * Updates a digest with a long.
+     * @param md the digest to update
+     * @param l8 the long
+     */
+    private static void digest(MessageDigest md, long l8) {
+        md.update((byte) (l8 & 0xff));
+        md.update((byte) (l8 >> 8));
+        md.update((byte) (l8 >> 16));
+        md.update((byte) (l8 >> 24));
+        md.update((byte) (l8 >> 32));
+        md.update((byte) (l8 >> 40));
+        md.update((byte) (l8 >> 48));
+        md.update((byte) (l8 >> 56));
+    }
+
+    /**
+     * Updates a digest with an object.
+     * @param md the digest to update
+     * @param obj the object
+     */
+    private static void digest(MessageDigest md, Object obj) {
+        if (obj == null) {
+            return;
+        }
+        try (ObjectOutput out = new ObjectOutputStream(new TagOutputStream(md))) {
+            out.writeObject(obj);
+        } catch (IOException ex) {
+            // ignore close exception
+        }
+        // ignore close exception
+    }
+
+    /**
+     * Computes the uuid.
+     * @param md the message digest used to compute the hash
+     * @return the eTag as a type 5 uuid
+     */
+    private static UUID computeUUID(MessageDigest md) {
+        byte[] sha1Bytes = md.digest();
+        sha1Bytes[6] &= 0x0f;  /* clear version        */
+        sha1Bytes[6] |= 0x50;  /* set to version 5     */
+        sha1Bytes[8] &= 0x3f;  /* clear variant        */
+        sha1Bytes[8] |= 0x80;  /* set to IETF variant  */
+
+        // SHA generates 160 bytes; truncate to 128
+        long msb = 0;
+        //assert data.length == 16 || data.length == 20;
+        for (int i = 0; i < 8; i++) {
+            msb = (msb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        long lsb = 0;
+        for (int i = 8; i < 16; i++) {
+            lsb = (lsb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        return new UUID(msb, lsb);
+    }
+
+    /**
+     * A marker interface for objects that can be digested.
+     */
+    public interface Digestible {
+        /**
+         * Updates a digest with this variable.
+         * @param digester the digester to update
+         * @return true if this digestible actually contributed to the digest
+         */
+        boolean digest(Digester digester);
+    }
+
+    /**
+     * A type 5 uuid is namespace + sha1; namespace in our case is a uuid.
+     * Two instances of digesters built with the same namespace will produce the same UUIDs from the
+     * same inputs.
+     * @param namespace the uuid namespace
+     */
+    public Digester(UUID namespace) {
+        nsuid = namespace == null? TEMP_NS : namespace;
+        md = createDigest();
+        // inject namespace
+        digest(md, nsuid);
+    }
+
+    /**
+     * A copy ctor base.
+     * @param lnsuid the namespace uid
+     * @param lmd the message digest
+     */
+    private Digester(UUID lnsuid, MessageDigest lmd)  {
+        this.nsuid = lnsuid;
+        this.md = lmd;
+    }
+
+    /**
+     * Default ctor.
+     * The created digester uses the class loading time as seed for its namespace; this means 2 instances of digester
+     * built in different JVM instances will *NOT* produce the same UUIDs for the same input. Typical use is in
+     * a non-persistent scenario, to verify an instance of an object has not been modified by checking
+     * its digested UUID remained the same.
+     * To get stable UUID computation across time and space in Digester usable in persistent scenario,
+     * you *NEED* to use a namespace-based digester using {@link Digester(UUID)}, uuid that is easily created
+     * using {@link UUID#nameUUIDFromBytes(byte[])} from any name/uri you might desire.
+     */
+    public Digester() {
+        this(null);
+    }
+
+    /**
+     * @return a clone of this instance
+     */
+    public Digester copy() {
+        try {
+            return new Digester(nsuid, (MessageDigest) md.clone());
+        } catch (CloneNotSupportedException ex) {
+            return null;
+        }
+    }
+
+    /**
+     * Computes the version tag from this digester.
+     * <p>This uses the current message digest state and resets it.
+     * @return the type 5 uuid
+     */
+    public UUID getUUID() {
+        UUID uuid = computeUUID(md);
+        md.reset();
+        digest(nsuid);
+        return uuid;
+    }
+
+    /**
+     * Updates the digest with a boolean.
+     * @param b the boolean
+     * @return this digester
+     */
+    public Digester digest(boolean b) {
+        md.update((byte) (b? 1 : 0));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a char.
+     * @param c the char
+     * @return this digester
+     */
+    public Digester digest(char c) {
+        md.update((byte) (c & 0xff));
+        md.update((byte) (c >> 8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a bytes array.
+     * @param bytes the bytes
+     * @return this digester
+     */
+    public Digester digest(byte[] bytes) {
+        if (bytes != null) {
+            md.update(bytes);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with an integer.
+     * @param i4 the int
+     * @return this digester
+     */
+    public Digester digest(int i4) {
+        md.update((byte) (i4 & 0xff));
+        md.update((byte) (i4 >> 8));
+        md.update((byte) (i4 >> 16));
+        md.update((byte) (i4 >> 24));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a long.
+     * @param l8 the long
+     * @return this digester
+     */
+    public Digester digest(long l8) {
+        digest(md, l8);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a double.
+     * @param f8 the double
+     * @return this digester
+     */
+    public Digester digest(double f8) {
+        digest(md, Double.doubleToRawLongBits(f8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a date.
+     * @param date the date
+     * @return this digester
+     */
+    public Digester digest(Date date) {
+        if (date != null) {
+            digest(md, date.getTime());
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a string.
+     * @param str the string
+     * @return this digester
+     */
+    public Digester digest(String str) {
+        if (str != null) {
+            final int sz = str.length();
+            for(int i = 0; i < sz; ++i) {
+                digest(str.charAt(i));
+            }
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uid the uuid
+     * @return this digester
+     */
+    public Digester digest(UUID uid) {
+        digest(md, uid);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uri the uri
+     * @return this digester
+     */
+    public Digester digest(URI uri) {
+        digest(md, uri.toString());
+        return this;
+    }
+
+    /**
+     * Updates the digest with an object that describes how it digests.
+     * @param digestible the object
+     * @return this digester
+     */
+    public Digester digest(Digestible digestible) {
+        if (digestible != null) {
+            digestible.digest(this);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a stream.
+     * @param input the stream
+     * @return this digester
+     */
+    public Digester digest(InputStream input) {
+        if (input != null) {
+            digest(md, input);
+        }
+        return this;
+    }
+    /**
+     * Updates the digest with any (serializable) object.
+     * @param obj the object
+     * @return this digester
+     */
+    public Digester digest(Object obj) {
+        if (obj instanceof Digestible) {
+            return digest((Digestible) obj);
+        }
+        if (obj instanceof UUID) {
+            return digest((UUID) obj);
+        }
+        if (obj instanceof URI) {
+            return digest((URI) obj);
+        }
+        if (obj instanceof String) {
+            return digest((String) obj);
+        }
+        if (obj instanceof Date) {
+            return digest((Date) obj);
+        }
+        if (obj instanceof Integer) {
+            return digest(((Integer) obj).intValue());
+        }
+        if (obj instanceof Long) {
+            return digest(((Long) obj).longValue());
+        }
+        if (obj instanceof Double) {
+            return digest(((Double) obj).doubleValue());
+        }
+        if (obj instanceof Boolean) {
+            return digest(((Boolean) obj).booleanValue());
+        }
+        if (obj instanceof Character) {
+            return digest(((Character) obj).charValue());
+        }
+        if (obj instanceof Short) {
+            return digest(((Short) obj).intValue());
+        }
+        if (obj instanceof Float) {
+            return digest(((Float) obj).doubleValue());
+        }
+        if (obj != null && obj.getClass().isArray()) {
+           int sz  = java.lang.reflect.Array.getLength(obj);
+           for(int i = 0; i < sz; ++i) {
+               digest(java.lang.reflect.Array.get(obj, i));
+           }

Review Comment:
   Good catch! corrected.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187474104


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/SerializationProxy.java:
##########
@@ -0,0 +1,614 @@
+/*
+ * 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.properties;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.Externalizable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Executable;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.apache.hadoop.hive.metastore.properties.Serializer.SERIALIZER;
+
+/**
+ * The serialization proxy template.
+ * <p>
+ * This allows a class that defines final members to be made serializable in an easy way.
+ * The class <em>must</em> implement:
+ * <ul>
+ * <li>a constructor that takes a DataInput (or derived class) as parameter</li>
+ * <li>a write method that takes a DataOutput (or derived class) as parameter</li>
+ * </ul>
+ * <p>
+ *   One should consider the constructor as being potentially fed with an invalid stream so
+ *   all usual checks of a public constructor should apply.
+ * </p>
+ * Standard usage is to add the Serializable interface implementation through the following 2 methods:
+ * <code>
+ * private Object writeReplace() throws ObjectStreamException {
+ *     return new SerializationProxy&lt;TheClass&gt;(this);
+ * }
+ * private void readObject(ObjectInputStream in)throws IOException,ClassNotFoundException{
+ *     throw new InvalidObjectException("proxy required");
+ * }
+ * </code>
+ * @param <T> the serializable object type
+ */
+public class SerializationProxy<T extends Serializable> implements Externalizable {
+  /** Serial version. */
+  private static final long serialVersionUID = 202212281757L;
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(SerializationProxy.class);
+  /** The map of class names to types. */
+  private static final ConcurrentMap<String, Type<?>> TYPES = new ConcurrentHashMap<>();
+  /** The list of registered pre-defined classes. */
+  private static final List<Type<?>> REGISTERED = new ArrayList<>();
+  /** A thread local context used for arguments passing during serialization/de-serialization. */
+  private static final ThreadLocal<Object[]> EXTRA_ARGUMENTS = new ThreadLocal<>();
+
+  /** The type of instance being read or written. */
+  private transient Type<T> type = null;
+  /** The instance being read or written. */
+  private transient T proxied = null;
+
+  /**
+   * Wraps any error that may occur whilst using reflective calls.
+   */
+  public static class ProxyException extends RuntimeException {
+    public ProxyException(Throwable cause) {
+      super(cause);
+    }
+
+    public ProxyException(String msg) {
+      super(msg);
+    }
+
+    /**
+     * Convert an exception to a VDBRuntimeException.
+     * @param cause the exception to convert
+     * @return the wrapping CubeException
+     */
+    public static ProxyException convert(Throwable cause) {
+      if (cause instanceof ProxyException) {
+        return (ProxyException) cause;
+      } else {
+        return new ProxyException(cause);
+      }
+    }
+  }
+
+  /**
+   * Constructor called from proxify.writeReplace().
+   * @param proxify the instance to proxy
+   */
+  @SuppressWarnings("unchecked")
+  public SerializationProxy(T proxify) {
+    Class<T> clazz = (Class<T>) proxify.getClass();
+    type = (Type<T>) TYPES.computeIfAbsent(clazz.getName(), this::createType);
+    proxied = proxify;
+  }
+
+  /**
+   * Default constructor.
+   */
+  public SerializationProxy() {
+    // do nothing
+  }
+
+  /**
+   * Sets the extra-arguments as a thread local context.
+   * <p>Used to pass extra arguments o constructors/write methods.</p>
+   * @param o the arguments
+   */
+  public static void setExtraArguments(Object[] o) {
+    if (null == o) {
+      EXTRA_ARGUMENTS.remove();
+    } else {
+      EXTRA_ARGUMENTS.set(o);
+    }
+  }
+
+  /**
+   * Gets the extra-arguments to ctor/write executable stored in a thread local context.
+   * @return the arguments
+   */
+  public static Object[] getExtraArguments() {
+    return EXTRA_ARGUMENTS.get();
+  }
+
+  /**
+   * Swaps the thread local context.
+   * <p>This may be used to stack up contexts during cascading calls.</p>
+   * @param newArgs the new arguments
+   * @return the down-stack caller arguments
+   */
+  public static Object[] swapExtraArguments(Object[] newArgs) {
+    Object[] previous = EXTRA_ARGUMENTS.get();
+    setExtraArguments(newArgs);
+    return previous;
+  }
+
+  /**
+   * Unloads the proxy.
+   */
+  public static void unload() {
+    EXTRA_ARGUMENTS.remove();
+    TYPES.clear();
+  }
+
+  /**
+   * Registers a pre-defined class (known to be used throughout the whole application).
+   * @param <T> the type
+   * @param slot the slot number
+   * @param clazz the class
+   */
+  public static <T extends Serializable> void registerType(final int slot, Class<T> clazz) {
+    synchronized (REGISTERED) {
+      Type<T> ntype = new Type<>(clazz);
+      ntype.slot = slot;
+      if (slot >= 255) {
+        throw new IllegalArgumentException(ntype + "@" + slot + ": can not register more than 254 types");
+      }
+      List<Type<?>> types = REGISTERED;
+      while (types.size() <= slot) {
+        types.add(null);
+      }
+      if (types.get(slot) != null) {
+        throw new IllegalArgumentException(ntype + "@" + slot + ": slot already used by " + types.get(slot));
+      }
+      types.set(slot, ntype);
+      TYPES.put(clazz.getName(), ntype);
+    }
+  }
+
+  /**
+   * Called by serialization after readExternal.
+   * @return the proxied instance
+   * @throws IOException for signature compliance
+   */
+  public Object readResolve() throws IOException {
+    return proxied;
+  }
+
+  @Override
+  public void readExternal(ObjectInput in) throws IOException {
+    long serial = in.readLong();
+    if (serial != serialVersionUID) {
+      throw new ProxyException("invalid serial version, got " + serial +", expected " + serialVersionUID);
+    }
+    type = readType(in);
+    proxied = type.proxyNew(in);
+  }
+
+  @Override
+  public void writeExternal(ObjectOutput out) throws IOException {
+    out.writeLong(serialVersionUID);
+    writeType(type, out);
+    type.proxyWrite(proxied, out);
+  }
+
+  /**
+   * Converts a serializable object to an array of bytes.
+   * @param serializable the object to serialize
+   * @param args the proxy arguments
+   * @return the array of bytes
+   * @throws ProxyException on any underlying error
+   */
+  public static byte[] toBytes(Serializable serializable, Object... args) {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(512);
+    final Object[] stack = SerializationProxy.swapExtraArguments(args);
+    try (ObjectOutput oos = new ObjectOutputStream(bos)) {
+      oos.writeObject(serializable);
+      oos.flush();
+      return bos.toByteArray();
+    } catch (IOException xany) {
+      throw ProxyException.convert(xany);
+    } finally {
+      SerializationProxy.swapExtraArguments(stack);
+    }
+  }
+
+  /**
+   * Materialize a serializable object from an array of bytes.
+   * @param bytes the bytes
+   * @param args the proxy arguments
+   * @return the object
+   * @throws ProxyException on any underlying error
+   */
+  public static <T extends Serializable> T fromBytes(byte[] bytes, Object... args) {
+    ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+    final Object[] stack = SerializationProxy.swapExtraArguments(args);
+    try (ObjectInput ois = new ObjectInputStream(bis)) {
+      return (T) ois.readObject();
+    } catch (IOException | ClassNotFoundException | ClassCastException xany) {
+      throw ProxyException.convert(xany);
+    } finally {
+      SerializationProxy.swapExtraArguments(stack);
+    }
+  }
+
+  /**
+   * Saves an object to persistent storage.
+   * @param file the file to write to
+   * @param persist the object to serialize
+   * @param args the proxy constructor arguments
+   * @return true if successful, false if file is null
+   * @throws ProxyException in case of low level error
+   */
+  public static boolean write(File file, Serializable persist, Object... args) {
+    return SERIALIZER.write(file, persist, args);
+  }
+
+  /**
+   * Saves an object to persistent storage.
+   * @param out the stream to write to
+   * @param persist the object to serialize
+   * @param args the proxy write method arguments
+   * @return true if successful, false if file is null
+   * @throws ProxyException in case of low level error
+   */
+  public static boolean write(OutputStream out, Serializable persist, Object... args) {
+    return SERIALIZER.write(out, persist, args);
+  }
+
+  /**
+   * Loads an object from the persistent storage.
+   * @param file the file to read from
+   * @param args the proxy arguments
+   * @return the object or null if file is null
+   * @throws ProxyException in case of low level error
+   */
+  public static Serializable read(File file, Object... args) {
+    return SERIALIZER.read(file, args);
+  }
+
+  /**
+   * Loads an object from the persistent storage.
+   * @param in the stream to read from
+   * @param args the proxy arguments
+   * @return the object or null if file is null
+   * @throws ProxyException in case of low level error
+   */
+  public static <T extends Serializable> T read(InputStream in, Object... args) {
+    return SERIALIZER.read(in, args);
+  }
+
+  /**
+   * Creates a Type using a class name.
+   * @param cname the class name
+   * @return a type instance
+   * @throws ProxyException on any underlying error
+   */
+   protected Type<T> createType(String cname) {
+    try {
+      @SuppressWarnings("unchecked")
+      Class<T> clazz = (Class<T>) Class.forName(cname);
+      return new Type<>(clazz);
+    } catch (ClassNotFoundException xnotfound) {
+      throw ProxyException.convert(xnotfound);
+    }
+  }
+
+  /**
+   * When writing out this instance, write down the canonical class name it proxifies.
+   * @param out the output
+   * @throws IOException if things go wrong
+   */
+  protected void writeType(Type<?> type, DataOutput out) throws IOException {
+    int slot = type.getSlot();
+    out.write(slot);
+    if (slot == 255) {
+      out.writeUTF(type.getTargetName());
+    }
+  }
+
+  /**
+   * When reading an instance, fetch the type through the canonical class name that was persisted.
+   * @param in the input
+   * @throws IOException on read error
+   * @throws ProxyException if class was expected to be registered but can not be found
+   */
+  @SuppressWarnings("unchecked")
+  protected Type<T> readType(DataInput in) throws IOException {
+    final Type<T> type;
+    String className = "?";
+    int slot = (int) in.readByte() & 0xff;
+    if (slot == 255) {
+      className = in.readUTF();
+      type = (Type<T>) TYPES.computeIfAbsent(className, this::createType);
+    } else if (slot < REGISTERED.size()) {
+      type = (Type<T>) REGISTERED.get(slot);
+    } else {
+      type = null;
+    }
+    if (type == null) {
+      throw new ProxyException("can not resolve class @ " + slot +", " + className);
+    }
+    return type;
+  }
+
+  /**
+   * Encapsulates the mandatory constructor and write methods for a given proxified class.
+   * @param <T> the proxified class
+   */
+  protected static class Type<T extends Serializable> {
+    private final Constructor<T>[] ctors;
+    private final Method[] writes;
+    private transient int slot = 255;
+
+    /**
+     * Creates a new instance of type.
+     * @param clazz the proxified class
+     */
+    public Type(Class<T> clazz) {
+        ctors = typeConstructors(clazz);
+        writes = typeWrites(clazz);
+    }
+
+    /**
+     * The slot number if the class is registered.
+     * @return the slot number, 255 means not-registered
+     */
+    public int getSlot() {
+      return slot;
+    }
+
+    /**
+     * @return the target class
+     */
+    public String getTargetName() {
+      // there is always at least one ctor
+      return ctors[0].getDeclaringClass().getName();
+    }
+
+    /**
+     * Compare parameter signatures of executables.
+     * @param lhs left-hand side
+     * @param rhs right-hand side
+     * @return 0 if equal, +/- 1 if left &lt;/&gt; than right
+     */
+    private static int compareSignatures(Executable lhs, Executable rhs) {
+      return compareSignatures(lhs.getParameterTypes(), rhs.getParameterTypes());
+    }
+
+    /**
+     * Compare executables parameter signatures.
+     * @param lhs left-hand side executable
+     * @param rhs right-hand side executable
+     * @return 0 if equal, +/- 1 if left &lt;/&gt; than right
+     */
+    private static int compareSignatures(Class<?>[] lhs, Class<?>[] rhs) {
+      if (lhs.length < rhs.length) {
+        return -1;
+      }
+      if (lhs.length > rhs.length) {
+        return 1;
+      }
+      int cmp = 0;
+      // lhs.length == rhs.length
+      final int length = lhs.length;
+      for (int p = 0; p < length; ++p) {
+        Class<?> actual = lhs[p];
+        Class<?> formal = rhs[p];
+        if (formal != null && actual != null && !formal.isAssignableFrom(actual)) {
+          // if formal parameter is primitive and actual argument is compatible
+          int dist;
+          if (formal.isPrimitive() && (dist = CONVERTIBLES.get(formal).indexOf(actual)) >= 0) {
+            cmp +=  dist;
+            continue;
+          }
+          dist = formal.getName().compareTo(actual.getName());
+          if (dist != 0) {
+            return cmp * (length - p);

Review Comment:
   Should be `dist *  (length - p)`, good catch.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174276284


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/CachingPropertyStore.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.properties;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * A property map store using a pull-thru cache.
+ * Before a map is returned, a check against the stored corresponding digest is performed to avoid
+ * using stale data.
+ */
+public class CachingPropertyStore extends PropertyStore {
+  private static final int MAP_CACHE = 64;
+  protected  SoftCache<String, PropertyMap> maps;
+  protected PropertyStore store;
+  public CachingPropertyStore(PropertyStore wrap) {
+    this(wrap, MAP_CACHE);
+  }
+
+  public CachingPropertyStore(PropertyStore wrap, int capacity) {
+    store = wrap;
+    maps = new SoftCache<>(capacity, false);
+  }
+  public void clearCache() {
+    maps.clear();
+  }
+
+  @Override public UUID fetchDigest(String mapKey) {
+    return store.fetchDigest(mapKey);
+  }
+
+  @Override
+  public Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter) {
+    return store.selectDigest(keyPrefix, keyFilter);
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, final Function<String, PropertySchema> getSchema) {
+    synchronized(this) {
+      PropertyMap map = maps.compute(mapKey, mapsCompute(mapKey, getSchema));
+      // we always return a copy of the properties in the cache
+      return map != null? map.copy() : null;
+    }
+  }
+
+  BiFunction<String, PropertyMap, PropertyMap> mapsCompute(String string, Function<String, PropertySchema> getSchema) {

Review Comment:
   The map key is a string; it is generic by design since the PropertyManager is responsible for encoding/decoding whatever semantic is needed (schema resolution included). I tried with a class storing the various segments but it added lots of complications and constraints for very little abstraction gains (if any).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174273634


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -167,6 +167,39 @@ const string HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__"
 const string HIVE_FILTER_FIELD_TABLE_NAME = "hive_filter_field_tableName__"
 const string HIVE_FILTER_FIELD_TABLE_TYPE = "hive_filter_field_tableType__"
 
+/** Table maintenance operation type. */
+enum MaintenanceOpType {
+    COMPACTION = 1,
+    SNAPSHOT_EXPIRY = 2,
+    STATS_REBUILD= 3,
+    MV_BUILD= 4,

Review Comment:
   These are intended as descriptors for data life-cycle management features.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1513910016

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [85 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171890976


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Digester.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.properties;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Date;
+import java.util.UUID;
+
+/**
+ * Helper class that creates a type 5 uuid.
+ * <p>This is computed from a set of updates using a SHA-1 message digest massaged into an UUID.
+ * see <a href="https://en.wikipedia.org/wiki/Universally_unique_identifier">...</a>
+ */
+public class Digester {
+    /** The Namespace uuid. */
+    private final UUID nsuid;
+    /** The digest used to compute the UUID. */
+    private final MessageDigest md;
+    /** A default namespace based on the class loading time. */
+    private static final UUID TEMP_NS;
+    static {
+        MessageDigest md = createDigest();
+        digest(md, System.currentTimeMillis());
+        TEMP_NS = computeUUID(md);
+    }
+
+    /**
+     * Allows to update the message digest from an object.
+     */
+    private static class TagOutputStream extends OutputStream {
+        /** The digest to update. */
+        private final MessageDigest md;
+
+        /**
+         * Sole ctor.
+         * @param md the message digester
+         */
+        TagOutputStream(MessageDigest md) {
+            this.md = md;
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            md.update((byte) b);
+        }
+
+        @Override
+        public void write(byte[] b, int off, int len) throws IOException {
+            md.update(b, off, len);
+        }
+    }
+
+    /**
+     * @return a SHA-1 message digest
+     */
+    private static MessageDigest createDigest() {
+        MessageDigest md;
+        try {
+            md = MessageDigest.getInstance("SHA-1");
+        } catch (NoSuchAlgorithmException nsae) {
+            throw new InternalError("SHA not supported");
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with a uuid.
+     * @param md the digest to update
+     * @param uid the uuid
+     */
+    private static MessageDigest digest(MessageDigest md, UUID uid) {
+        if (uid != null) {
+            long msb = uid.getMostSignificantBits();
+            digest(md, msb);
+            long lsb = uid.getLeastSignificantBits();
+            digest(md, lsb);
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with an input stream.
+     * @param md the digest to update
+     * @param input the input to consume
+     * @throws IllegalStateException if an io exception occurs
+     */
+    private static void digest(MessageDigest md, InputStream input) {
+        try (OutputStream out = new TagOutputStream(md)) {
+            byte[] buffer = new byte[1024];
+            int read;
+            while ((read = input.read(buffer, 0, 1024)) >= 0) {
+                out.write(buffer, 0, read);
+            }
+        } catch(IOException xio) {
+            throw new IllegalStateException(xio);
+        }
+    }
+
+
+    /**
+     * Updates a digest with a long.
+     * @param md the digest to update
+     * @param l8 the long
+     */
+    private static void digest(MessageDigest md, long l8) {
+        md.update((byte) (l8 & 0xff));
+        md.update((byte) (l8 >> 8));
+        md.update((byte) (l8 >> 16));
+        md.update((byte) (l8 >> 24));
+        md.update((byte) (l8 >> 32));
+        md.update((byte) (l8 >> 40));
+        md.update((byte) (l8 >> 48));
+        md.update((byte) (l8 >> 56));
+    }
+
+    /**
+     * Updates a digest with an object.
+     * @param md the digest to update
+     * @param obj the object
+     */
+    private static void digest(MessageDigest md, Object obj) {
+        if (obj == null) {
+            return;
+        }
+        try (ObjectOutput out = new ObjectOutputStream(new TagOutputStream(md))) {
+            out.writeObject(obj);
+        } catch (IOException ex) {
+            // ignore close exception
+        }
+        // ignore close exception
+    }
+
+    /**
+     * Computes the uuid.
+     * @param md the message digest used to compute the hash
+     * @return the eTag as a type 5 uuid
+     */
+    private static UUID computeUUID(MessageDigest md) {
+        byte[] sha1Bytes = md.digest();
+        sha1Bytes[6] &= 0x0f;  /* clear version        */
+        sha1Bytes[6] |= 0x50;  /* set to version 5     */
+        sha1Bytes[8] &= 0x3f;  /* clear variant        */
+        sha1Bytes[8] |= 0x80;  /* set to IETF variant  */
+
+        // SHA generates 160 bytes; truncate to 128
+        long msb = 0;
+        //assert data.length == 16 || data.length == 20;
+        for (int i = 0; i < 8; i++) {
+            msb = (msb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        long lsb = 0;
+        for (int i = 8; i < 16; i++) {
+            lsb = (lsb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        return new UUID(msb, lsb);
+    }
+
+    /**
+     * A marker interface for objects that can be digested.
+     */
+    public interface Digestible {
+        /**
+         * Updates a digest with this variable.
+         * @param digester the digester to update
+         * @return true if this digestible actually contributed to the digest
+         */
+        boolean digest(Digester digester);
+    }
+
+    /**
+     * A type 5 uuid is namespace + sha1; namespace in our case is a uuid.
+     * Two instances of digesters built with the same namespace will produce the same UUIDs from the
+     * same inputs.
+     * @param namespace the uuid namespace
+     */
+    public Digester(UUID namespace) {
+        nsuid = namespace == null? TEMP_NS : namespace;
+        md = createDigest();
+        // inject namespace
+        digest(md, nsuid);
+    }
+
+    /**
+     * A copy ctor base.
+     * @param lnsuid the namespace uid
+     * @param lmd the message digest
+     */
+    private Digester(UUID lnsuid, MessageDigest lmd)  {
+        this.nsuid = lnsuid;
+        this.md = lmd;
+    }
+
+    /**
+     * Default ctor.
+     * The created digester uses the class loading time as seed for its namespace; this means 2 instances of digester
+     * built in different JVM instances will *NOT* produce the same UUIDs for the same input. Typical use is in
+     * a non-persistent scenario, to verify an instance of an object has not been modified by checking
+     * its digested UUID remained the same.
+     * To get stable UUID computation across time and space in Digester usable in persistent scenario,
+     * you *NEED* to use a namespace-based digester using {@link Digester(UUID)}, uuid that is easily created
+     * using {@link UUID#nameUUIDFromBytes(byte[])} from any name/uri you might desire.
+     */
+    public Digester() {
+        this(null);
+    }
+
+    /**
+     * @return a clone of this instance
+     */
+    public Digester copy() {
+        try {
+            return new Digester(nsuid, (MessageDigest) md.clone());
+        } catch (CloneNotSupportedException ex) {
+            return null;
+        }
+    }
+
+    /**
+     * Computes the version tag from this digester.
+     * <p>This uses the current message digest state and resets it.
+     * @return the type 5 uuid
+     */
+    public UUID getUUID() {
+        UUID uuid = computeUUID(md);
+        md.reset();
+        digest(nsuid);
+        return uuid;
+    }
+
+    /**
+     * Updates the digest with a boolean.
+     * @param b the boolean
+     * @return this digester
+     */
+    public Digester digest(boolean b) {
+        md.update((byte) (b? 1 : 0));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a char.
+     * @param c the char
+     * @return this digester
+     */
+    public Digester digest(char c) {
+        md.update((byte) (c & 0xff));
+        md.update((byte) (c >> 8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a bytes array.
+     * @param bytes the bytes
+     * @return this digester
+     */
+    public Digester digest(byte[] bytes) {
+        if (bytes != null) {
+            md.update(bytes);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with an integer.
+     * @param i4 the int
+     * @return this digester
+     */
+    public Digester digest(int i4) {
+        md.update((byte) (i4 & 0xff));
+        md.update((byte) (i4 >> 8));
+        md.update((byte) (i4 >> 16));
+        md.update((byte) (i4 >> 24));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a long.
+     * @param l8 the long
+     * @return this digester
+     */
+    public Digester digest(long l8) {
+        digest(md, l8);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a double.
+     * @param f8 the double
+     * @return this digester
+     */
+    public Digester digest(double f8) {
+        digest(md, Double.doubleToRawLongBits(f8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a date.
+     * @param date the date
+     * @return this digester
+     */
+    public Digester digest(Date date) {
+        if (date != null) {
+            digest(md, date.getTime());
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a string.
+     * @param str the string
+     * @return this digester
+     */
+    public Digester digest(String str) {
+        if (str != null) {
+            final int sz = str.length();
+            for(int i = 0; i < sz; ++i) {
+                digest(str.charAt(i));
+            }
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uid the uuid
+     * @return this digester
+     */
+    public Digester digest(UUID uid) {
+        digest(md, uid);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uri the uri
+     * @return this digester
+     */
+    public Digester digest(URI uri) {
+        digest(md, uri.toString());
+        return this;
+    }
+
+    /**
+     * Updates the digest with an object that describes how it digests.
+     * @param digestible the object
+     * @return this digester
+     */
+    public Digester digest(Digestible digestible) {
+        if (digestible != null) {
+            digestible.digest(this);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a stream.
+     * @param input the stream
+     * @return this digester
+     */
+    public Digester digest(InputStream input) {
+        if (input != null) {
+            digest(md, input);
+        }
+        return this;
+    }
+    /**
+     * Updates the digest with any (serializable) object.
+     * @param obj the object
+     * @return this digester
+     */
+    public Digester digest(Object obj) {
+        if (obj instanceof Digestible) {

Review Comment:
   Do you think this can be better implemented in the switch case?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] TuroczyX commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "TuroczyX (via GitHub)" <gi...@apache.org>.
TuroczyX commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1158306692


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/JdoPropertyStore.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.commons.jexl3.JexlException;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertySchema;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * Implementation of the property store delegating persistence to a (jdo) raw store.
+ */
+public class JdoPropertyStore extends PropertyStore {
+  /** The jdo objects store. */
+  private final RawStore objectStore;
+
+  /**
+   * Basic ctor.
+   * @param store the object store
+   */
+  public JdoPropertyStore(RawStore store) {
+    this.objectStore = store;
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.getProperties(mapKey, getPropertyMapFunction(null, getSchema));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public Map<String, PropertyMap> selectProperties(final String keyPrefix, Predicate<String> keyFilter, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.selectProperties(keyPrefix, getPropertyMapFunction(keyFilter, getSchema));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public UUID fetchDigest(String mapKey) {
+    try {
+      return objectStore.getProperties(mapKey, (mm) -> UUID.fromString(mm.getPropertyValue()));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter) {
+    try {
+      return objectStore.selectProperties(keyPrefix, (mm) -> {
+        if (keyFilter == null || keyFilter.test(mm.getPropertykey())) {
+          return UUID.fromString(mm.getPropertyValue());
+        }
+        return null;
+      });
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public void saveProperties(Iterator<Map.Entry<String, PropertyMap>> save) {
+    // will run the super method in a transaction
+    try {
+      objectStore.runInTransaction(()-> super.saveProperties(save));
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  protected void saveProperties(String mapKey, PropertyMap map) {
+    try {
+      if (map.isDropped()) {
+        objectStore.dropProperties(mapKey);
+      } else {
+        objectStore.putProperties(mapKey, map.getDigest().toString(), null, serialize(map));
+      }
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override public boolean dropProperties(String mapKey) {
+   try {
+     return objectStore.dropProperties(mapKey);
+   } catch (MetaException e) {
+     throw new PropertyException(e);
+   }
+  }
+
+  @Override public boolean renameProperties(String mapKey, String newKey) {
+    try {
+      return objectStore.renameProperties(mapKey, newKey);
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+  /**
+   * Creates a function that transforms an MMetastoreDBProperties into a PropertyMap.
+   * @param keyFilter a map key filtering predicate that will make the function return null if test fails
+   * @param getSchema the function that solves a schema from a key
+   * @return a function
+   */
+  Function<MMetastoreDBProperties, PropertyMap> getPropertyMapFunction(final Predicate<String> keyFilter, final Function<String, PropertySchema> getSchema) {
+    return (mm) -> {
+      final String key = mm.getPropertykey();

Review Comment:
   Why the final keyword is necessary here in this method? 



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1525991626

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [112 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1506955116

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [85 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1522141536

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [105 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175368339


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -5192,4 +5192,25 @@ public List<WriteEventInfo> getAllWriteEventInfo(GetAllWriteEventInfoRequest req
   public AbortCompactResponse abortCompactions(AbortCompactionRequest request) throws TException{
     return client.abort_Compactions(request);
   }
+
+  @Override
+  public boolean setProperties(String nameSpace, Map<String, String> properties) throws TException {
+    PropertySetRequest psr = new PropertySetRequest();
+    psr.setNameSpace(nameSpace);
+    psr.setPropertyMap(properties);
+    return client.set_properties(psr);
+  }
+
+  @Override
+  public Map<String, Map<String, String>> getProperties(String nameSpace, String mapPrefix, String mapPredicate, String... selection) throws TException {
+    PropertyGetRequest request = new PropertyGetRequest();
+    request.setNameSpace(nameSpace);
+    request.setMapPrefix(mapPrefix);
+    request.setMapPredicate(mapPredicate);
+    if (selection != null && selection.length > 0) {
+      request.setMapSelection(Arrays.asList(selection));
+    }
+    PropertyGetResponse response = client.get_properties(request);

Review Comment:
   I like being able to put a breakpoint before I return (just in case...).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187398678


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyMap.java:
##########
@@ -0,0 +1,466 @@
+/*
+ * 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.properties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * A property map pertaining to a given object type (cluster, database, table).
+ * <p>
+ *   Maps follow a copy-on-write scheme gated by a dirty flag (avoid copy of a dirty map). This allows
+ *   sharing their content (the inner properties map) with guaranteed isolation and thread safety.
+ * </p>
+ */
+public class PropertyMap implements Serializable {

Review Comment:
   I check the SerializationProxy, looks like we use something like `public Abc(DataInput input, Object... args)` to deserialize the object from byte array, and use `write(DataOutput out)` to serialize the instance, so my idea is that, we'd better to have these two methods as a template for SerializationProxy.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1204596260


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -0,0 +1,268 @@
+/* * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonSyntaxException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyManager;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.Source;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * The property  cli servlet.
+ */
+public class PropertyServlet extends HttpServlet {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyServlet.class);
+  /** The object store. */
+  private final RawStore objectStore;

Review Comment:
   Thank you for pointing out this big mistake; corrected in last commit (HMSHandler like).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1561662757

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [103 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1204628282


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyStore.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.properties;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * The PropertyStore is the persistent container of property maps.
+ * Maps are addressed in the store by their key -  their name prepended by their manager&quot;s namespace.
+ */
+
+public abstract class PropertyStore {
+  /**
+   * Fetches a property map.
+   * @param mapKey the map key
+   * @param getSchema the method to retrieve a schema if the map needs to be created
+   * @return the item property map
+   */
+  public abstract PropertyMap fetchProperties(String mapKey, Function<String, PropertySchema> getSchema);
+
+
+  /**
+   * Fetches a map of property maps.
+   * @param keyPrefix the map key prefix
+   * @param keyFilter a filter for map keys
+   * @param getSchema the method to retrieve a schema if the map needs to be created
+   * @return the map of property map
+   */
+  public abstract Map<String, PropertyMap> selectProperties(final String keyPrefix, Predicate<String> keyFilter, Function<String, PropertySchema> getSchema);
+  /**
+   * Fetches a property map digest.
+   * @param mapKey the map key
+   * @return the item property map
+   */
+  public abstract UUID fetchDigest(String mapKey);
+
+  /**
+   * Fetches a map of property maps digest keyed by their name.
+   * @param keyPrefix the map key prefix
+   * @param keyFilter a filter for map keys
+   * @return the map of property map digests
+   */
+  public abstract Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter);
+
+    /**
+     * Persists a property map.
+     * @param mapKey the map key
+     * @param map the map instance
+     */
+  protected abstract void saveProperties(String mapKey, PropertyMap map);
+
+
+  /**
+   * Drops a property map.
+   * @param mapKey the map key
+   */
+  protected abstract boolean dropProperties(String mapKey);
+
+  /**
+   * Renames a property map.
+   * @param mapKey the map source key
+   * @param newKey the new target key
+   */
+  public abstract boolean renameProperties(String mapKey, String newKey);
+
+  /**
+   * Persists an iterator property map.
+   * <p>May be useful to override to use one transaction.</p>
+   * @param save the iterator on pairs for map key, property map
+   */
+  public void saveProperties(Iterator<Map.Entry<String, PropertyMap>> save) {
+    while(save.hasNext()) {
+      Map.Entry<String, PropertyMap> pair = save.next();
+      PropertyMap map = pair.getValue();
+      if (map != null) {
+        saveProperties(pair.getKey(), map);
+      } else {
+        dropProperties(pair.getKey());
+      }
+    }
+  }
+
+  /**
+   * Serializes a map as a byte array.
+   * @param map the (nonnull) map to write
+   * @return the byte array
+   */
+  public byte[] serialize(PropertyMap map) {
+    return SerializationProxy.toBytes(map);
+  }
+
+  /**
+   * Deserializes a map from a byte array.
+   * @param bytes the byte array
+   * @return the (nonnull) oroperty map
+   */
+  public PropertyMap deserialize(byte[] bytes, Function<String, PropertySchema> getSchema) {
+    return SerializationProxy.fromBytes(bytes, this, getSchema);
+  }
+
+  /**
+   * Default ctor.
+   */
+  protected PropertyStore() {
+  }
+
+  /**
+   * A non-persistent store, for tests mainly.
+   */
+  public static class Transient extends PropertyStore {

Review Comment:
   Done.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1548583334

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [110 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1568095518

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [93 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1177651769


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();

Review Comment:
   If we need statefulness/session and a non-transient PropertyManager, it becomes necessary to keep isolation.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171297701


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -2769,6 +2802,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   GetPartitionsByNamesResult get_partitions_by_names_req(1:GetPartitionsByNamesRequest req)
                         throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
+    // retrieve properties
+    PropertyGetResponse get_properties(1:PropertyGetRequest req);

Review Comment:
   Should we declare the throwing exception of these two methods? I'm afraid the Thrift would throw transport exception instead if things go wrong.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174278261


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/SoftCache.java:
##########
@@ -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.hadoop.hive.metastore.properties;
+
+import java.lang.ref.SoftReference;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * A soft referenced cache.
+ * <p>
+ * The actual cache is held through a soft reference, allowing it to be GCed under memory pressure.</p>
+ * <p>
+ * This class is <em>not</em> thread-safe.</p>
+ * @param <K> the cache key entry type
+ * @param <V> the cache key value type
+ */
+public class SoftCache<K, V> {
+    /** The default cache capacity. */
+    private static final int CACHE_CAPACITY = 64;
+    /** The default cache load factor. */
+    private static final float LOAD_FACTOR = 0.75f;

Review Comment:
   We can certainly expose its default properties; where/how ?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1572970172

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [90 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187402296


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java:
##########
@@ -113,7 +113,9 @@
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;

Review Comment:
   nit: unused import



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1539365660

   The latest change overall looks good to me. @saihemanth-cloudera @nrg4878 Could you please take a look if have a chance? Thanks!


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1550113320

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [110 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1203899245


##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/properties/HMSServletTest.java:
##########
@@ -0,0 +1,278 @@
+/* * 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.properties;
+
+import com.google.gson.Gson;
+import com.nimbusds.jose.JWSAlgorithm;
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.JWSSigner;
+import com.nimbusds.jose.crypto.RSASSASigner;
+import com.nimbusds.jose.jwk.RSAKey;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.SignedJWT;
+
+import com.github.tomakehurst.wiremock.junit.WireMockRule;
+import org.apache.commons.httpclient.methods.PostMethod;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.PropertyServlet;
+import org.apache.hadoop.hive.metastore.ServletSecurity;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpMethod;
+import org.apache.commons.httpclient.NameValuePair;
+import org.apache.commons.httpclient.methods.DeleteMethod;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.commons.httpclient.methods.PutMethod;
+import org.apache.commons.httpclient.methods.StringRequestEntity;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.eclipse.jetty.server.Server;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletResponse;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.get;
+import static com.github.tomakehurst.wiremock.client.WireMock.ok;
+
+public class HMSServletTest extends HMSTestBase {
+
+  private static String baseDir = System.getProperty("basedir");
+  private static final File jwtAuthorizedKeyFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-authorized-key.json");
+  private static final File jwtUnauthorizedKeyFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-unauthorized-key.json");
+  private static final File jwtVerificationJWKSFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-verification-jwks.json");
+
+  public static final String USER_1 = "USER_1";
+
+  private static final int MOCK_JWKS_SERVER_PORT = 8089;
+  @ClassRule
+  public static final WireMockRule MOCK_JWKS_SERVER = new WireMockRule(MOCK_JWKS_SERVER_PORT);
+  // the url part
+  private static final String CLI = "hmscli";
+  Server servletServer = null;
+  int sport = -1;
+
+
+  @Override protected int createServer(Configuration conf) throws Exception {
+    // need store before server for servlet
+    if (objectStore == null) {
+      MetastoreConf.setVar(conf, MetastoreConf.ConfVars.PROPERTIES_SERVLET_AUTH, "JWT");
+      MetastoreConf.setVar(conf, MetastoreConf.ConfVars.THRIFT_METASTORE_AUTHENTICATION_JWT_JWKS_URL,
+          "http://localhost:" + MOCK_JWKS_SERVER_PORT + "/jwks");
+      MOCK_JWKS_SERVER.stubFor(get("/jwks")
+          .willReturn(ok()
+              .withBody(Files.readAllBytes(jwtVerificationJWKSFile.toPath()))));
+      boolean inited = createStore(conf);
+      LOG.info("MetaStore store initialization " + (inited ? "successful" : "failed"));
+    }
+    if (servletServer == null) {
+      servletServer = PropertyServlet.startServer(conf, CLI, objectStore);
+      if (!servletServer.isStarted()) {
+        Assert.fail("http server did not start");
+      }
+      sport = servletServer.getURI().getPort();
+    }
+    return sport;
+  }
+
+  /**
+   * Stops the server.
+   * @param port the server port
+   */
+  @Override protected void stopServer(int port) throws Exception {
+    if (servletServer != null) {
+      servletServer.stop();
+      servletServer = null;
+      sport = -1;
+    }
+  }
+
+  @Override
+  protected JSonClient createClient(Configuration conf, int sport) throws Exception {
+    URL url = new URL("http://hive@localhost:" + sport + "/" + CLI + "/" + NS);
+    String jwt = generateJWT();
+    return new JSonClient(jwt, url);
+  }
+
+  private String generateJWT()  throws Exception {
+    return generateJWT(USER_1, jwtAuthorizedKeyFile.toPath(), TimeUnit.MINUTES.toMillis(5));
+  }
+
+  private static String generateJWT(String user, Path keyFile, long lifeTimeMillis) throws Exception {
+    RSAKey rsaKeyPair = RSAKey.parse(new String(java.nio.file.Files.readAllBytes(keyFile), StandardCharsets.UTF_8));
+    // Create RSA-signer with the private key
+    JWSSigner signer = new RSASSASigner(rsaKeyPair);
+    JWSHeader header = new JWSHeader
+        .Builder(JWSAlgorithm.RS256)
+        .keyID(rsaKeyPair.getKeyID())
+        .build();
+    Date now = new Date();
+    Date expirationTime = new Date(now.getTime() + lifeTimeMillis);
+    JWTClaimsSet claimsSet = new JWTClaimsSet.Builder()
+        .jwtID(UUID.randomUUID().toString())
+        .issueTime(now)
+        .issuer("auth-server")
+        .subject(user)
+        .expirationTime(expirationTime)
+        .claim("custom-claim-or-payload", "custom-claim-or-payload")
+        .build();
+    SignedJWT signedJWT = new SignedJWT(header, claimsSet);
+    // Compute the RSA signature
+    signedJWT.sign(signer);
+    return signedJWT.serialize();
+  }
+
+
+  /**
+   * A property client that uses http as transport.
+   */
+  public static class JSonClient implements PropertyClient {
+    private final URL url;
+    private String jwt = null;
+    JSonClient(String token, URL url) {
+      this.jwt = token;
+      this.url = url;
+    }
+
+    public boolean setProperties(Map<String, String> properties) {
+      try {
+        clientCall(jwt, url, "PUT", properties);
+        return true;
+      } catch(IOException xio) {
+        return false;
+      }
+    }
+
+    public Map<String, Map<String, String>> getProperties(String mapPrefix, String mapPredicate, String... selection) throws IOException {
+      Map<String, Object> args = new TreeMap<>();
+      args.put("prefix", mapPrefix);
+      if (mapPredicate != null) {
+        args.put("predicate", mapPredicate);
+      }
+      if (selection != null && selection.length > 0) {
+        args.put("selection", selection);
+      }
+      try {
+        Object result = clientCall(jwt, url, "POST", args);
+        return result instanceof Map? (Map<String, Map<String, String>>) result : null ;
+      } catch(IOException xio) {
+        return null;
+      }
+    }
+  }
+
+  @Test
+  public void testJSONServlet() throws Exception {
+    URL url = new URL("http://hive@localhost:" + sport + "/" + CLI + "/" + NS);
+    Map<String, String> json = Collections.singletonMap("method", "echo");
+    String jwt = generateJWT();
+    Object response = clientCall(jwt, url, "POST", json);
+    Assert.assertNotNull(response);
+    Assert.assertEquals(json, response);
+
+    response = clientCall(null, url, "POST", json);
+    Assert.assertNull(response);
+  }
+
+  @Test
+  public void testProperties1() throws Exception {
+      runOtherProperties1(client);
+
+  }
+
+  @Test
+  public void testProperties0() throws Exception {
+      runOtherProperties0(client);
+  }
+
+  @Test
+  public void testPropertiesOtherClient() throws Exception {
+    HttpClient client = new HttpClient();
+    HttpMethod method = new PostMethod("http://hive@localhost:" + sport + "/" + CLI + "/" + NS);
+
+    String jwt = generateJWT();
+    method.addRequestHeader("Authorization","Bearer " + jwt);
+    method.addRequestHeader("Content-Type", "application/json");
+    method.addRequestHeader("Accept", "application/json");
+
+    String msgBody = "{\"method\":\"echo\"}";

Review Comment:
   nit: can we add a unit test for method `selectProperties`



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1573883514

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [90 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1528182945

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [115 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171281173


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property mamanger class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized(NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor != null) {
+      try {
+        return ctor.newInstance(namespace, store);
+      } catch(Exception xany) {
+        LOGGER.error("property manager creation failed "+ namespace, xany);
+      }
+    } else {
+      LOGGER.error("no such property manager namespace is declared " + namespace);
+    }
+    return null;
+  }
+
+  /**
+   * JEXL adapter.
+   */
+  public static class MapWrapper implements JexlContext {
+    PropertyMap map;
+    MapWrapper(PropertyMap map) {
+      this.map = map;
+    }
+
+    public Object get(String p) {
+      return map.getPropertyValue(p);
+    }
+
+    @Override
+    public void set(String name, Object value) {
+      map.putProperty(name, value);
+    }
+
+    @Override
+    public boolean has(String name) {
+      return map.getTypeOf(name) != null;
+    }
+  }
+
+  /**
+   * Creates a manager instance.
+   * @param store the store instance which must use an appropriate property map factory (probably use createMap).
+   */
+  protected PropertyManager(String ns, PropertyStore store) {
+    this.namespace = ns;
+    this.store = store;
+  }
+
+  /**
+   * Saves all pending updates to store.
+   */
+  public void commit() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      if (!dirtyMaps.isEmpty()) {
+        store.saveProperties(dirtyMaps.entrySet().iterator());
+        dirtyMaps.clear();
+      }
+    }
+  }
+
+  /**
+   * Forget all pending updates.
+   */
+  public void rollback() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      dirtyMaps.clear();
+    }
+  }
+
+  /**
+   * Imports a set of default values into this store&quot;s schema.
+   * The properties should be of the form schema_name.property_name=value.
+   * Note that this implies the manager has at least one known property map schema.
+   * @param importsp the properties
+   */
+  public void importDefaultValues(Properties importsp) {
+    importsp.forEach((k, v)->{
+      String importName = k.toString();
+      final int dotPosition = importName.indexOf(".");
+      if (dotPosition > 0) {
+        String schemaName = importName.substring(0, dotPosition);
+        PropertySchema schema = getSchema(schemaName);
+        if (schema != null) {
+          String propertyName = importName.substring(dotPosition + 1);
+          schema.setDefaultValue(propertyName, v);
+        }
+      }
+    });
+  }
+
+  /**
+   * Imports a set of property values.
+   * @param map the properties key=value
+   */
+  public void setProperties(Properties map) {
+    map.forEach((k, v)-> setProperty(k.toString(), v));
+  }
+
+  /**
+   * Injects a set of properties.
+   * If the value is null, the property is removed.
+   * @param map the map of properties to inject.
+   */
+  public void setProperties(Map<String, Object> map) {
+    map.forEach(this::setProperty);
+  }
+
+  /**
+   * Sets a property value.
+   * @param key the property key
+   * @param value the property value or null to unset
+   */
+  public void setProperty(String key, Object value) {
+    setProperty(splitKey(key), value);
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or null if not assigned
+   */
+  public Object getProperty(String key) {
+    return getProperty(splitKey(key));
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or the schema default value if not assigned
+   */
+  public Object getPropertyValue(String key) {
+    return getPropertyValue(splitKey(key));
+  }
+
+  /**
+   * Splits a property key into its fragments.
+   * @param key the property key
+   * @return the key fragments
+   */
+  protected String[] splitKey(String key) {
+    String[] splits = key.split("(?<!\\\\)\\.");
+    if (splits.length < 1) {
+      splits = new String[]{key};
+    }
+    return splits;
+  }
+
+  /**
+   * Gets a schema by name.
+   * <p>Only used by {@link #importDefaultValues(Properties)}</p>
+   * @param name schema name
+   * @return the schema instance, null if no such schema is known
+   */
+  public PropertySchema getSchema(String name) {
+    return null;
+  }
+
+  /**
+   * Determines the schema from the property key fragments.
+   * @param keys the key fragments
+   * @return the schema, {@link PropertySchema#NONE} if no such schema is known
+   */
+  protected PropertySchema schemaOf(String[] keys) {
+    return PropertySchema.NONE;
+  }
+
+  /**
+   * @param keys property key fragments
+   * @return number of fragments composing the map name in the fragments array
+   */
+  protected int getMapNameLength(String[] keys) {
+    return keys.length - 1;
+  }
+
+  /**
+   * Compose a property map key from a property map name.
+   * @param name the property map name, may be null or empty
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String name) {
+    StringBuilder strb = new StringBuilder(namespace);
+    if (name != null && !name.isEmpty()){
+      strb.append('.');
+      strb.append(name);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Extract a property map name from a property map key.
+   * @param key property map key
+   * @return the property map name
+   */
+  protected String mapName(String key) {
+    int dot = key.indexOf('.');
+    return dot > 0? key.substring(dot + 1) : key;
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the key fragments
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String[] keys) {
+    return mapKey(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the property key fragments
+   * @param maxkl the maximum number of fragments in the map key
+   * @return the property key used by the store
+   */
+  protected String mapKey(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    // shortest map key is namespace
+    StringBuilder strb = new StringBuilder(namespace);
+    for(int k = 0; k < Math.min(maxkl, keys.length - 1); ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @return the property name
+   */
+  protected String propertyName(String[] keys) {
+    return propertyName(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @param maxkl the maximum number of fragments in the map name
+   * @return the property name
+   */
+  protected String propertyName(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    if (keys.length <= maxkl) {
+      return keys[keys.length - 1];
+    }
+    StringBuilder strb = new StringBuilder(keys[maxkl]);
+    for(int k = maxkl + 1; k < keys.length; ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or null if none was assigned
+   */
+  public Object getProperty(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    if (map == null) {
+      map = store.fetchProperties(mapKey, null);
+    }
+    if (map != null) {
+      return map.getProperty(propertyName(keys));
+    }
+    return null;
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or the default schema value if not assigned
+   */
+  public Object getPropertyValue(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    PropertySchema schema = schemaOf(keys);
+    if (map == null) {
+      map = store.fetchProperties(mapKey, s->schema);
+    }
+    String propertyName = propertyName(keys);
+    if (map != null) {
+      return map.getPropertyValue(propertyName);
+    }
+    if (schema != null) {
+      return schema.getDefaultValue(propertyName);
+    }
+    return null;
+  }
+
+  /**
+   * Drops a property map.
+   * @param mapName the map name
+   * @return true if the properties may exist, false if they did nots
+   */
+  public boolean dropProperties(String mapName) {
+    final String mapKey = mapKey(mapName);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null && Objects.equals(PropertyMap.DROPPED, dirtyMap.getDigest())) {
+      map = dirtyMap;
+    } else {
+      // is is stored ?
+      UUID digest = store.fetchDigest(mapKey);
+      // not stored nor cached, nothing to do
+      if (digest == null) {
+        return false;
+      }
+      map = new PropertyMap(store, schemaOf(splitKey(mapName + ".*")), PropertyMap.DROPPED);
+      synchronized (dirtyMaps) {
+        dirtyMaps.put(mapName, map);
+      }
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapName, map);
+    }
+    return false;
+  }
+
+  /**
+   * Sets a property value.
+   * @param keys the key fragments
+   * @param value the new value or null if mapping should be removed
+   */
+  public void setProperty(String[] keys, Object value) {
+    // find schema from key (length)
+    PropertySchema schema = schemaOf(keys);
+    String mapKey = mapKey(keys);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null) {
+      map = dirtyMap;
+    } else {
+      // is is stored ?
+      map = store.fetchProperties(mapKey, s->schema);
+      if (map == null) {
+        // remove a value from a non persisted map, noop
+        if (value == null) {
+          return;
+        }
+        map = new PropertyMap(store, schema);
+      }
+    }
+    // map is not null
+    String propertyName = propertyName(keys);
+    if (value != null) {
+      map.putProperty(propertyName, value);
+    } else {
+      map.removeProperty(propertyName);
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapKey, map);
+    }
+  }
+
+  public Map<String, PropertyMap> selectProperties(String namePrefix, String predicateStr, String... projectStr) {
+    return selectProperties(namePrefix, predicateStr,
+        projectStr == null
+            ? Collections.emptyList()
+            : Arrays.asList(projectStr));
+  }
+
+  /**
+   * Selects a set of properties.
+   * @param mapPrefix the map name prefix
+   * @param selector the selector/transformer function
+   * @return the map of property maps keyed by their name
+   */
+  public Map<String, PropertyMap> selectProperties(String mapPrefix, Function<PropertyMap, PropertyMap> selector) {
+    final String mapKey = mapKey(mapPrefix);
+    final Map<String, PropertyMap> selected = store.selectProperties(mapKey,null, k->schemaOf(splitKey(k)) );
+    final Map<String, PropertyMap> maps = new TreeMap<>();
+    final Function<PropertyMap, PropertyMap> transform = selector == null? Function.identity() : selector;
+    selected.forEach((k, p) -> {
+      final PropertyMap dirtyMap = dirtyMaps.get(k);
+      final PropertyMap map = transform.apply(dirtyMap == null ? p.copy() : dirtyMap.copy());
+      if (map != null && !map.isEmpty()) {
+        maps.put(mapName(k), map);
+      }
+    });
+    return maps;
+  }
+  public Map<String, PropertyMap> selectProperties(String prefix, String predicateStr, List<String> projectStr) {
+    final JexlExpression predicate;
+    try {
+      predicate = JEXL.createExpression(predicateStr);
+    } catch(JexlException.Parsing xparse) {
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug(predicateStr, xparse);
+      }
+      throw xparse;
+    }
+    final Function<PropertyMap, PropertyMap> transform = (map)->{
+      MapWrapper wrapped = new MapWrapper(map);
+      Object result;
+      try {
+        result = predicate.evaluate(wrapped);

Review Comment:
   Not sure if it will bring potential security problem provided that the `predicateStr` can be arbitrary.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171701352


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java:
##########
@@ -4393,4 +4393,27 @@ ReplicationMetricList getReplicationMetrics(GetReplicationMetricsRequest
 
   AbortCompactResponse abortCompactions(AbortCompactionRequest request) throws TException;
 
+  /**
+   * Sets properties.
+   * @param nameSpace the property store namespace
+   * @param properties a map keyed by property path mapped to property values
+   * @return true if successful, false otherwise
+   * @throws TException
+   */
+  default boolean setProperties(String nameSpace, Map<String, String> properties) throws TException {
+    throw new UnsupportedOperationException();

Review Comment:
   I think this definition is incorrect. We should only declare the method here and any other dummy clients or clients that don't support this method should define this as unsupported.
   Currently, SessionHiveMetaStoreClient would throw this exception. Can you please implement these methods in SessionHiveMetastoreClient as well?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171900856


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property mamanger class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized(NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor != null) {
+      try {
+        return ctor.newInstance(namespace, store);
+      } catch(Exception xany) {
+        LOGGER.error("property manager creation failed "+ namespace, xany);
+      }
+    } else {
+      LOGGER.error("no such property manager namespace is declared " + namespace);

Review Comment:
   IMHO, When namespace creation is failed we should throw an error instead of silently logging the message.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175629261


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property mamanger class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized(NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor != null) {
+      try {
+        return ctor.newInstance(namespace, store);
+      } catch(Exception xany) {
+        LOGGER.error("property manager creation failed "+ namespace, xany);
+      }
+    } else {
+      LOGGER.error("no such property manager namespace is declared " + namespace);

Review Comment:
   Made it throw errors.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174258092


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();

Review Comment:
   Because the property manager owns the  property schema(s) and different implementations are expected; we 'replace' modifying the db schema by allowing different property manager namespaces and manager implementations. Next feature that needs to persist metadata properties may create its own (think partitions metadata).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174259729


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MMetastoreDBProperties.java:
##########
@@ -21,6 +21,8 @@ public class MMetastoreDBProperties {
   private String propertyKey;
   private String propertyValue;
   private String description;
+  private byte[] propertyContent;

Review Comment:
   We actually do add a column which is a blob; the update scripts are lacking. But newly created schema do have one more column in that table.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1518824613

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [92 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174257187


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property mamanger class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized(NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor != null) {
+      try {
+        return ctor.newInstance(namespace, store);
+      } catch(Exception xany) {
+        LOGGER.error("property manager creation failed "+ namespace, xany);
+      }
+    } else {
+      LOGGER.error("no such property manager namespace is declared " + namespace);
+    }
+    return null;
+  }
+
+  /**
+   * JEXL adapter.
+   */
+  public static class MapWrapper implements JexlContext {
+    PropertyMap map;
+    MapWrapper(PropertyMap map) {
+      this.map = map;
+    }
+
+    public Object get(String p) {
+      return map.getPropertyValue(p);
+    }
+
+    @Override
+    public void set(String name, Object value) {
+      map.putProperty(name, value);
+    }
+
+    @Override
+    public boolean has(String name) {
+      return map.getTypeOf(name) != null;
+    }
+  }
+
+  /**
+   * Creates a manager instance.
+   * @param store the store instance which must use an appropriate property map factory (probably use createMap).
+   */
+  protected PropertyManager(String ns, PropertyStore store) {
+    this.namespace = ns;
+    this.store = store;
+  }
+
+  /**
+   * Saves all pending updates to store.
+   */
+  public void commit() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      if (!dirtyMaps.isEmpty()) {
+        store.saveProperties(dirtyMaps.entrySet().iterator());
+        dirtyMaps.clear();
+      }
+    }
+  }
+
+  /**
+   * Forget all pending updates.
+   */
+  public void rollback() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      dirtyMaps.clear();
+    }
+  }
+
+  /**
+   * Imports a set of default values into this store&quot;s schema.
+   * The properties should be of the form schema_name.property_name=value.
+   * Note that this implies the manager has at least one known property map schema.
+   * @param importsp the properties
+   */
+  public void importDefaultValues(Properties importsp) {
+    importsp.forEach((k, v)->{
+      String importName = k.toString();
+      final int dotPosition = importName.indexOf(".");
+      if (dotPosition > 0) {
+        String schemaName = importName.substring(0, dotPosition);
+        PropertySchema schema = getSchema(schemaName);
+        if (schema != null) {
+          String propertyName = importName.substring(dotPosition + 1);
+          schema.setDefaultValue(propertyName, v);
+        }
+      }
+    });
+  }
+
+  /**
+   * Imports a set of property values.
+   * @param map the properties key=value
+   */
+  public void setProperties(Properties map) {
+    map.forEach((k, v)-> setProperty(k.toString(), v));
+  }
+
+  /**
+   * Injects a set of properties.
+   * If the value is null, the property is removed.
+   * @param map the map of properties to inject.
+   */
+  public void setProperties(Map<String, Object> map) {
+    map.forEach(this::setProperty);
+  }
+
+  /**
+   * Sets a property value.
+   * @param key the property key
+   * @param value the property value or null to unset
+   */
+  public void setProperty(String key, Object value) {
+    setProperty(splitKey(key), value);
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or null if not assigned
+   */
+  public Object getProperty(String key) {
+    return getProperty(splitKey(key));
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or the schema default value if not assigned
+   */
+  public Object getPropertyValue(String key) {
+    return getPropertyValue(splitKey(key));
+  }
+
+  /**
+   * Splits a property key into its fragments.
+   * @param key the property key
+   * @return the key fragments
+   */
+  protected String[] splitKey(String key) {
+    String[] splits = key.split("(?<!\\\\)\\.");
+    if (splits.length < 1) {
+      splits = new String[]{key};
+    }
+    return splits;
+  }
+
+  /**
+   * Gets a schema by name.
+   * <p>Only used by {@link #importDefaultValues(Properties)}</p>
+   * @param name schema name
+   * @return the schema instance, null if no such schema is known
+   */
+  public PropertySchema getSchema(String name) {
+    return null;
+  }
+
+  /**
+   * Determines the schema from the property key fragments.
+   * @param keys the key fragments
+   * @return the schema, {@link PropertySchema#NONE} if no such schema is known
+   */
+  protected PropertySchema schemaOf(String[] keys) {
+    return PropertySchema.NONE;
+  }
+
+  /**
+   * @param keys property key fragments
+   * @return number of fragments composing the map name in the fragments array
+   */
+  protected int getMapNameLength(String[] keys) {
+    return keys.length - 1;
+  }
+
+  /**
+   * Compose a property map key from a property map name.
+   * @param name the property map name, may be null or empty
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String name) {
+    StringBuilder strb = new StringBuilder(namespace);
+    if (name != null && !name.isEmpty()){
+      strb.append('.');
+      strb.append(name);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Extract a property map name from a property map key.
+   * @param key property map key
+   * @return the property map name
+   */
+  protected String mapName(String key) {
+    int dot = key.indexOf('.');
+    return dot > 0? key.substring(dot + 1) : key;
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the key fragments
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String[] keys) {
+    return mapKey(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the property key fragments
+   * @param maxkl the maximum number of fragments in the map key
+   * @return the property key used by the store
+   */
+  protected String mapKey(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    // shortest map key is namespace
+    StringBuilder strb = new StringBuilder(namespace);
+    for(int k = 0; k < Math.min(maxkl, keys.length - 1); ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @return the property name
+   */
+  protected String propertyName(String[] keys) {
+    return propertyName(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @param maxkl the maximum number of fragments in the map name
+   * @return the property name
+   */
+  protected String propertyName(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    if (keys.length <= maxkl) {
+      return keys[keys.length - 1];
+    }
+    StringBuilder strb = new StringBuilder(keys[maxkl]);
+    for(int k = maxkl + 1; k < keys.length; ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or null if none was assigned
+   */
+  public Object getProperty(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    if (map == null) {
+      map = store.fetchProperties(mapKey, null);
+    }
+    if (map != null) {
+      return map.getProperty(propertyName(keys));
+    }
+    return null;
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or the default schema value if not assigned
+   */
+  public Object getPropertyValue(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    PropertySchema schema = schemaOf(keys);
+    if (map == null) {
+      map = store.fetchProperties(mapKey, s->schema);
+    }
+    String propertyName = propertyName(keys);
+    if (map != null) {
+      return map.getPropertyValue(propertyName);
+    }
+    if (schema != null) {
+      return schema.getDefaultValue(propertyName);
+    }
+    return null;
+  }
+
+  /**
+   * Drops a property map.
+   * @param mapName the map name
+   * @return true if the properties may exist, false if they did nots
+   */
+  public boolean dropProperties(String mapName) {
+    final String mapKey = mapKey(mapName);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null && Objects.equals(PropertyMap.DROPPED, dirtyMap.getDigest())) {
+      map = dirtyMap;
+    } else {
+      // is is stored ?
+      UUID digest = store.fetchDigest(mapKey);
+      // not stored nor cached, nothing to do
+      if (digest == null) {
+        return false;
+      }
+      map = new PropertyMap(store, schemaOf(splitKey(mapName + ".*")), PropertyMap.DROPPED);
+      synchronized (dirtyMaps) {
+        dirtyMaps.put(mapName, map);
+      }
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapName, map);
+    }
+    return false;
+  }
+
+  /**
+   * Sets a property value.
+   * @param keys the key fragments
+   * @param value the new value or null if mapping should be removed
+   */
+  public void setProperty(String[] keys, Object value) {
+    // find schema from key (length)
+    PropertySchema schema = schemaOf(keys);
+    String mapKey = mapKey(keys);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null) {
+      map = dirtyMap;
+    } else {
+      // is is stored ?
+      map = store.fetchProperties(mapKey, s->schema);
+      if (map == null) {
+        // remove a value from a non persisted map, noop
+        if (value == null) {
+          return;
+        }
+        map = new PropertyMap(store, schema);
+      }
+    }
+    // map is not null
+    String propertyName = propertyName(keys);
+    if (value != null) {
+      map.putProperty(propertyName, value);
+    } else {
+      map.removeProperty(propertyName);
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapKey, map);
+    }
+  }
+
+  public Map<String, PropertyMap> selectProperties(String namePrefix, String predicateStr, String... projectStr) {
+    return selectProperties(namePrefix, predicateStr,
+        projectStr == null
+            ? Collections.emptyList()
+            : Arrays.asList(projectStr));
+  }
+
+  /**
+   * Selects a set of properties.
+   * @param mapPrefix the map name prefix
+   * @param selector the selector/transformer function
+   * @return the map of property maps keyed by their name
+   */
+  public Map<String, PropertyMap> selectProperties(String mapPrefix, Function<PropertyMap, PropertyMap> selector) {
+    final String mapKey = mapKey(mapPrefix);
+    final Map<String, PropertyMap> selected = store.selectProperties(mapKey,null, k->schemaOf(splitKey(k)) );
+    final Map<String, PropertyMap> maps = new TreeMap<>();
+    final Function<PropertyMap, PropertyMap> transform = selector == null? Function.identity() : selector;
+    selected.forEach((k, p) -> {
+      final PropertyMap dirtyMap = dirtyMaps.get(k);
+      final PropertyMap map = transform.apply(dirtyMap == null ? p.copy() : dirtyMap.copy());
+      if (map != null && !map.isEmpty()) {
+        maps.put(mapName(k), map);
+      }
+    });
+    return maps;
+  }
+  public Map<String, PropertyMap> selectProperties(String prefix, String predicateStr, List<String> projectStr) {
+    final JexlExpression predicate;
+    try {
+      predicate = JEXL.createExpression(predicateStr);
+    } catch(JexlException.Parsing xparse) {
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug(predicateStr, xparse);
+      }
+      throw xparse;
+    }
+    final Function<PropertyMap, PropertyMap> transform = (map)->{
+      MapWrapper wrapped = new MapWrapper(map);
+      Object result;
+      try {
+        result = predicate.evaluate(wrapped);

Review Comment:
   No issue there, JEXL is configured in a tight manner (no side-effects, accessible classes limited to the restricted).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1539010667

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [121 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1188093913


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,629 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.JexlScript;
+import org.apache.commons.jexl3.ObjectContext;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ * <p>The manager class needs to be registered with its namespace as key</p>
+ * <p>
+ *   Since a collection of properties are stored in a map, to avoid hitting the persistence store for each update
+ *   - which would mean rewriting the map multiple times - the manager keeps track of dirty maps whilst
+ *   serving as transaction manager. This way, when importing multiple properties targeting different elements (think
+ *   setting properties for different tables), each impacted map is only rewritten
+ *   once by the persistence layer during commit. This also allows multiple calls to participate to one transactions.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected final Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property manager class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   * @throws MetaException if the manager creation fails
+   * @throws NoSuchObjectException if the store is null or no constructor was declared
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) throws MetaException, NoSuchObjectException {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized (NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor == null) {
+      throw new NoSuchObjectException("no PropertyManager namespace is declared, namespace " + namespace);
+    }
+    if (store == null) {
+      throw new NoSuchObjectException("no PropertyStore exists " + namespace);
+    }
+    try {
+      return ctor.newInstance(namespace, store);
+    } catch (Exception xany) {
+      LOGGER.error("PropertyManager creation failed " + namespace, xany);
+      throw new MetaException("PropertyManager creation failed, namespace " + namespace);
+    }
+  }
+
+  /**
+   * JEXL adapter.
+   * <p>public for introspection.</p>
+   */
+  public static class MapWrapper implements JexlContext {
+    PropertyMap map;
+    MapWrapper(PropertyMap map) {
+      this.map = map;
+    }
+
+    public Object get(String p) {
+      return map.getPropertyValue(p);
+    }
+
+    @Override
+    public void set(String name, Object value) {
+      map.putProperty(name, value);
+    }
+
+    @Override
+    public boolean has(String name) {
+      return map.getTypeOf(name) != null;
+    }
+  }
+
+  /**
+   * Creates a manager instance.
+   * @param store the store instance which must use an appropriate property map factory (probably use createMap).
+   */
+  protected PropertyManager(String ns, PropertyStore store) {
+    this.namespace = ns;
+    this.store = store;
+  }
+
+  /**
+   * Saves all pending updates to store.
+   */
+  public void commit() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      if (!dirtyMaps.isEmpty()) {
+        store.saveProperties(dirtyMaps.entrySet().iterator());
+        dirtyMaps.clear();
+      }
+    }
+  }
+
+  /**
+   * Forget all pending updates.
+   */
+  public void rollback() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      dirtyMaps.clear();
+    }
+  }
+
+  /**
+   * Imports a set of default values into this store&quot;s schema.
+   * The properties should be of the form schema_name.property_name=value.
+   * Note that this implies the manager has at least one known property map schema.
+   * @param importsp the properties
+   */
+  public void importDefaultValues(Properties importsp) {
+    importsp.forEach((k, v)->{
+      String importName = k.toString();
+      final int dotPosition = importName.indexOf(".");
+      if (dotPosition > 0) {
+        String schemaName = importName.substring(0, dotPosition);
+        PropertySchema schema = getSchema(schemaName);
+        if (schema != null) {
+          String propertyName = importName.substring(dotPosition + 1);
+          schema.setDefaultValue(propertyName, v);
+        }
+      }
+    });
+  }
+
+  /**
+   * Imports a set of property values.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param map the properties key=value
+   */
+  public void setProperties(Properties map) {
+    map.forEach((k, v)-> setProperty(k.toString(), v));
+  }
+
+  /**
+   * Injects a set of properties.
+   * If the value is null, the property is removed.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param map the map of properties to inject.
+   */
+  public void setProperties(Map<String, ?> map) {
+    map.forEach(this::setProperty);
+  }
+
+  /**
+   * Sets a property value.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param key the property key
+   * @param value the property value or null to unset
+   */
+  public void setProperty(String key, Object value) {
+    setProperty(splitKey(key), value);
+  }
+
+  /**
+   * Runs a JEXL script using this manager as context.
+   * @param src the script source
+   * @return the script result
+   * @throws PropertyException if any error occurs in JEXL
+   */
+  public Object runScript(String src) throws PropertyException {
+    try {
+      JexlScript script = JEXL.createScript(src);
+      ObjectContext<PropertyManager> context = new ObjectContext<>(JEXL, this);
+      return script.execute(context);
+    } catch(JexlException je) {
+      throw new PropertyException("script failed", je);
+    }
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or null if not assigned
+   */
+  public Object getProperty(String key) {
+    return getProperty(splitKey(key));
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or the schema default value if not assigned
+   */
+  public Object getPropertyValue(String key) {
+    return getPropertyValue(splitKey(key));
+  }
+
+  /**
+   * Splits a property key into its fragments.
+   * @param key the property key
+   * @return the key fragments
+   */
+  protected String[] splitKey(String key) {
+    String[] splits = key.split("(?<!\\\\)\\.");
+    if (splits.length < 1) {
+      splits = new String[]{key};
+    }
+    return splits;
+  }
+
+  /**
+   * Gets a schema by name.
+   * <p>Only used by {@link #importDefaultValues(Properties)}</p>
+   * @param name schema name
+   * @return the schema instance, null if no such schema is known
+   */
+  public PropertySchema getSchema(String name) {
+    return null;
+  }
+
+  /**
+   * Determines the schema from the property key fragments.
+   * @param keys the key fragments
+   * @return the schema, {@link PropertySchema#NONE} if no such schema is known
+   */
+  protected PropertySchema schemaOf(String[] keys) {
+    return PropertySchema.NONE;
+  }
+
+  /**
+   * @param keys property key fragments
+   * @return number of fragments composing the map name in the fragments array
+   */
+  protected int getMapNameLength(String[] keys) {
+    return keys.length - 1;
+  }
+
+  /**
+   * Compose a property map key from a property map name.
+   * @param name the property map name, may be null or empty
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String name) {
+    StringBuilder strb = new StringBuilder(namespace);
+    if (name != null && !name.isEmpty()){
+      strb.append('.');
+      strb.append(name);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Extract a property map name from a property map key.
+   * @param key property map key
+   * @return the property map name
+   */
+  protected String mapName(String key) {
+    int dot = key.indexOf('.');
+    return dot > 0? key.substring(dot + 1) : key;
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the key fragments
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String[] keys) {
+    return mapKey(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the property key fragments
+   * @param maxkl the maximum number of fragments in the map key
+   * @return the property key used by the store
+   */
+  protected String mapKey(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    // shortest map key is namespace
+    StringBuilder strb = new StringBuilder(namespace);
+    for(int k = 0; k < Math.min(maxkl, keys.length - 1); ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @return the property name
+   */
+  protected String propertyName(String[] keys) {
+    return propertyName(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @param maxkl the maximum number of fragments in the map name
+   * @return the property name
+   */
+  protected String propertyName(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    if (keys.length <= maxkl) {
+      return keys[keys.length - 1];
+    }
+    StringBuilder strb = new StringBuilder(keys[maxkl]);
+    for(int k = maxkl + 1; k < keys.length; ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or null if none was assigned
+   */
+  public Object getProperty(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    if (map == null) {
+      map = store.fetchProperties(mapKey, null);
+    }
+    if (map != null) {
+      return map.getProperty(propertyName(keys));
+    }
+    return null;
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or the default schema value if not assigned
+   */
+  public Object getPropertyValue(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    PropertySchema schema = schemaOf(keys);
+    if (map == null) {
+      map = store.fetchProperties(mapKey, s->schema);
+    }
+    String propertyName = propertyName(keys);
+    if (map != null) {
+      return map.getPropertyValue(propertyName);
+    }
+    if (schema != null) {
+      return schema.getDefaultValue(propertyName);
+    }
+    return null;
+  }
+
+  /**
+   * Drops a property map.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param mapName the map name
+   * @return true if the properties may exist, false if they did nots
+   */
+  public boolean dropProperties(String mapName) {
+    final String mapKey = mapKey(mapName);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null && Objects.equals(PropertyMap.DROPPED, dirtyMap.getDigest())) {
+      map = dirtyMap;
+    } else {
+      // is it stored ?
+      UUID digest = store.fetchDigest(mapKey);
+      // not stored nor cached, nothing to do
+      if (digest == null) {
+        return false;
+      }
+      map = new PropertyMap(store, schemaOf(splitKey(mapName + ".*")), PropertyMap.DROPPED);
+      synchronized (dirtyMaps) {
+        dirtyMaps.put(mapName, map);
+      }
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapName, map);
+    }
+    return false;
+  }
+
+  /**
+   * Sets a property value.
+   * @param keys the key fragments
+   * @param value the new value or null if mapping should be removed
+   */
+  protected void setProperty(String[] keys, Object value) {
+    // find schema from key (length)
+    PropertySchema schema = schemaOf(keys);
+    String mapKey = mapKey(keys);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null) {
+      map = dirtyMap;
+    } else {
+      // is is stored ?
+      map = store.fetchProperties(mapKey, s->schema);
+      if (map == null) {
+        // remove a value from a non persisted map, noop
+        if (value == null) {
+          return;
+        }
+        map = new PropertyMap(store, schema);
+      }
+    }
+    // map is not null
+    String propertyName = propertyName(keys);
+    if (value != null) {
+      map.putProperty(propertyName, value);
+    } else {
+      map.removeProperty(propertyName);
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapKey, map);
+    }
+  }
+
+  /**
+   * Selects a set of properties.
+   * @param namePrefix the map name prefix
+   * @param predicateStr the condition selecting maps
+   * @param projectStr the projection property names or script
+   * @return the map of property maps keyed by their name
+   */
+  public Map<String, PropertyMap> selectProperties(String namePrefix, String predicateStr, String... projectStr) {
+    return selectProperties(namePrefix, predicateStr,
+        projectStr == null
+            ? Collections.emptyList()
+            : Arrays.asList(projectStr));
+  }
+
+  /**
+   * Selects a set of properties.
+   * @param namePrefix the map name prefix
+   * @param selector the selector/transformer function
+   * @return the map of property maps keyed by their name
+   */
+  public Map<String, PropertyMap> selectProperties(String namePrefix, Function<PropertyMap, PropertyMap> selector) {
+    final String mapKey = mapKey(namePrefix);
+    final Map<String, PropertyMap> selected = store.selectProperties(mapKey,null, k->schemaOf(splitKey(k)) );
+    final Map<String, PropertyMap> maps = new TreeMap<>();
+    final Function<PropertyMap, PropertyMap> transform = selector == null? Function.identity() : selector;
+    selected.forEach((k, p) -> {
+      final PropertyMap dirtyMap = dirtyMaps.get(k);

Review Comment:
   For every new select request, we always create a new PropertyManager, so I guess the `dirtyMaps` is always empty in this way.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1535349475

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [113 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187524702


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -0,0 +1,307 @@
+/* * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonSyntaxException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyManager;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.Source;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.Part;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * The property  cli servlet.
+ */
+public class PropertyServlet extends HttpServlet {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyServlet.class);
+  /** The object store. */
+  private final RawStore objectStore;
+  /** The security. */
+  private final ServletSecurity security;
+
+  PropertyServlet(Configuration configuration, RawStore store) {

Review Comment:
   We want other (no Thrift) clients to be able to communicate with the property api.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187292000


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/SerializationProxy.java:
##########
@@ -0,0 +1,614 @@
+/*
+ * 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.properties;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.Externalizable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Executable;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.apache.hadoop.hive.metastore.properties.Serializer.SERIALIZER;
+
+/**
+ * The serialization proxy template.
+ * <p>
+ * This allows a class that defines final members to be made serializable in an easy way.
+ * The class <em>must</em> implement:
+ * <ul>
+ * <li>a constructor that takes a DataInput (or derived class) as parameter</li>
+ * <li>a write method that takes a DataOutput (or derived class) as parameter</li>
+ * </ul>
+ * <p>
+ *   One should consider the constructor as being potentially fed with an invalid stream so
+ *   all usual checks of a public constructor should apply.
+ * </p>
+ * Standard usage is to add the Serializable interface implementation through the following 2 methods:
+ * <code>
+ * private Object writeReplace() throws ObjectStreamException {
+ *     return new SerializationProxy&lt;TheClass&gt;(this);
+ * }
+ * private void readObject(ObjectInputStream in)throws IOException,ClassNotFoundException{
+ *     throw new InvalidObjectException("proxy required");
+ * }
+ * </code>
+ * @param <T> the serializable object type
+ */
+public class SerializationProxy<T extends Serializable> implements Externalizable {
+  /** Serial version. */
+  private static final long serialVersionUID = 202212281757L;
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(SerializationProxy.class);
+  /** The map of class names to types. */
+  private static final ConcurrentMap<String, Type<?>> TYPES = new ConcurrentHashMap<>();
+  /** The list of registered pre-defined classes. */
+  private static final List<Type<?>> REGISTERED = new ArrayList<>();
+  /** A thread local context used for arguments passing during serialization/de-serialization. */
+  private static final ThreadLocal<Object[]> EXTRA_ARGUMENTS = new ThreadLocal<>();
+
+  /** The type of instance being read or written. */
+  private transient Type<T> type = null;
+  /** The instance being read or written. */
+  private transient T proxied = null;
+
+  /**
+   * Wraps any error that may occur whilst using reflective calls.
+   */
+  public static class ProxyException extends RuntimeException {
+    public ProxyException(Throwable cause) {
+      super(cause);
+    }
+
+    public ProxyException(String msg) {
+      super(msg);
+    }
+
+    /**
+     * Convert an exception to a VDBRuntimeException.
+     * @param cause the exception to convert
+     * @return the wrapping CubeException
+     */
+    public static ProxyException convert(Throwable cause) {
+      if (cause instanceof ProxyException) {
+        return (ProxyException) cause;
+      } else {
+        return new ProxyException(cause);
+      }
+    }
+  }
+
+  /**
+   * Constructor called from proxify.writeReplace().
+   * @param proxify the instance to proxy
+   */
+  @SuppressWarnings("unchecked")
+  public SerializationProxy(T proxify) {
+    Class<T> clazz = (Class<T>) proxify.getClass();
+    type = (Type<T>) TYPES.computeIfAbsent(clazz.getName(), this::createType);
+    proxied = proxify;
+  }
+
+  /**
+   * Default constructor.
+   */
+  public SerializationProxy() {
+    // do nothing
+  }
+
+  /**
+   * Sets the extra-arguments as a thread local context.
+   * <p>Used to pass extra arguments o constructors/write methods.</p>
+   * @param o the arguments
+   */
+  public static void setExtraArguments(Object[] o) {
+    if (null == o) {
+      EXTRA_ARGUMENTS.remove();
+    } else {
+      EXTRA_ARGUMENTS.set(o);
+    }
+  }
+
+  /**
+   * Gets the extra-arguments to ctor/write executable stored in a thread local context.
+   * @return the arguments
+   */
+  public static Object[] getExtraArguments() {
+    return EXTRA_ARGUMENTS.get();
+  }
+
+  /**
+   * Swaps the thread local context.
+   * <p>This may be used to stack up contexts during cascading calls.</p>
+   * @param newArgs the new arguments
+   * @return the down-stack caller arguments
+   */
+  public static Object[] swapExtraArguments(Object[] newArgs) {
+    Object[] previous = EXTRA_ARGUMENTS.get();
+    setExtraArguments(newArgs);
+    return previous;
+  }
+
+  /**
+   * Unloads the proxy.
+   */
+  public static void unload() {
+    EXTRA_ARGUMENTS.remove();
+    TYPES.clear();
+  }
+
+  /**
+   * Registers a pre-defined class (known to be used throughout the whole application).
+   * @param <T> the type
+   * @param slot the slot number
+   * @param clazz the class
+   */
+  public static <T extends Serializable> void registerType(final int slot, Class<T> clazz) {
+    synchronized (REGISTERED) {
+      Type<T> ntype = new Type<>(clazz);
+      ntype.slot = slot;
+      if (slot >= 255) {
+        throw new IllegalArgumentException(ntype + "@" + slot + ": can not register more than 254 types");
+      }
+      List<Type<?>> types = REGISTERED;
+      while (types.size() <= slot) {
+        types.add(null);
+      }
+      if (types.get(slot) != null) {
+        throw new IllegalArgumentException(ntype + "@" + slot + ": slot already used by " + types.get(slot));
+      }
+      types.set(slot, ntype);
+      TYPES.put(clazz.getName(), ntype);
+    }
+  }
+
+  /**
+   * Called by serialization after readExternal.
+   * @return the proxied instance
+   * @throws IOException for signature compliance
+   */
+  public Object readResolve() throws IOException {
+    return proxied;
+  }
+
+  @Override
+  public void readExternal(ObjectInput in) throws IOException {
+    long serial = in.readLong();
+    if (serial != serialVersionUID) {
+      throw new ProxyException("invalid serial version, got " + serial +", expected " + serialVersionUID);
+    }
+    type = readType(in);
+    proxied = type.proxyNew(in);
+  }
+
+  @Override
+  public void writeExternal(ObjectOutput out) throws IOException {
+    out.writeLong(serialVersionUID);
+    writeType(type, out);
+    type.proxyWrite(proxied, out);
+  }
+
+  /**
+   * Converts a serializable object to an array of bytes.
+   * @param serializable the object to serialize
+   * @param args the proxy arguments
+   * @return the array of bytes
+   * @throws ProxyException on any underlying error
+   */
+  public static byte[] toBytes(Serializable serializable, Object... args) {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(512);
+    final Object[] stack = SerializationProxy.swapExtraArguments(args);
+    try (ObjectOutput oos = new ObjectOutputStream(bos)) {
+      oos.writeObject(serializable);
+      oos.flush();
+      return bos.toByteArray();
+    } catch (IOException xany) {
+      throw ProxyException.convert(xany);
+    } finally {
+      SerializationProxy.swapExtraArguments(stack);
+    }
+  }
+
+  /**
+   * Materialize a serializable object from an array of bytes.
+   * @param bytes the bytes
+   * @param args the proxy arguments
+   * @return the object
+   * @throws ProxyException on any underlying error
+   */
+  public static <T extends Serializable> T fromBytes(byte[] bytes, Object... args) {
+    ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+    final Object[] stack = SerializationProxy.swapExtraArguments(args);
+    try (ObjectInput ois = new ObjectInputStream(bis)) {
+      return (T) ois.readObject();
+    } catch (IOException | ClassNotFoundException | ClassCastException xany) {
+      throw ProxyException.convert(xany);
+    } finally {
+      SerializationProxy.swapExtraArguments(stack);
+    }
+  }
+
+  /**
+   * Saves an object to persistent storage.
+   * @param file the file to write to
+   * @param persist the object to serialize
+   * @param args the proxy constructor arguments
+   * @return true if successful, false if file is null
+   * @throws ProxyException in case of low level error
+   */
+  public static boolean write(File file, Serializable persist, Object... args) {
+    return SERIALIZER.write(file, persist, args);
+  }
+
+  /**
+   * Saves an object to persistent storage.
+   * @param out the stream to write to
+   * @param persist the object to serialize
+   * @param args the proxy write method arguments
+   * @return true if successful, false if file is null
+   * @throws ProxyException in case of low level error
+   */
+  public static boolean write(OutputStream out, Serializable persist, Object... args) {
+    return SERIALIZER.write(out, persist, args);
+  }
+
+  /**
+   * Loads an object from the persistent storage.
+   * @param file the file to read from
+   * @param args the proxy arguments
+   * @return the object or null if file is null
+   * @throws ProxyException in case of low level error
+   */
+  public static Serializable read(File file, Object... args) {
+    return SERIALIZER.read(file, args);
+  }
+
+  /**
+   * Loads an object from the persistent storage.
+   * @param in the stream to read from
+   * @param args the proxy arguments
+   * @return the object or null if file is null
+   * @throws ProxyException in case of low level error
+   */
+  public static <T extends Serializable> T read(InputStream in, Object... args) {
+    return SERIALIZER.read(in, args);
+  }
+
+  /**
+   * Creates a Type using a class name.
+   * @param cname the class name
+   * @return a type instance
+   * @throws ProxyException on any underlying error
+   */
+   protected Type<T> createType(String cname) {
+    try {
+      @SuppressWarnings("unchecked")
+      Class<T> clazz = (Class<T>) Class.forName(cname);
+      return new Type<>(clazz);
+    } catch (ClassNotFoundException xnotfound) {
+      throw ProxyException.convert(xnotfound);
+    }
+  }
+
+  /**
+   * When writing out this instance, write down the canonical class name it proxifies.
+   * @param out the output
+   * @throws IOException if things go wrong
+   */
+  protected void writeType(Type<?> type, DataOutput out) throws IOException {
+    int slot = type.getSlot();
+    out.write(slot);
+    if (slot == 255) {
+      out.writeUTF(type.getTargetName());
+    }
+  }
+
+  /**
+   * When reading an instance, fetch the type through the canonical class name that was persisted.
+   * @param in the input
+   * @throws IOException on read error
+   * @throws ProxyException if class was expected to be registered but can not be found
+   */
+  @SuppressWarnings("unchecked")
+  protected Type<T> readType(DataInput in) throws IOException {
+    final Type<T> type;
+    String className = "?";
+    int slot = (int) in.readByte() & 0xff;
+    if (slot == 255) {
+      className = in.readUTF();
+      type = (Type<T>) TYPES.computeIfAbsent(className, this::createType);
+    } else if (slot < REGISTERED.size()) {
+      type = (Type<T>) REGISTERED.get(slot);
+    } else {
+      type = null;
+    }
+    if (type == null) {
+      throw new ProxyException("can not resolve class @ " + slot +", " + className);
+    }
+    return type;
+  }
+
+  /**
+   * Encapsulates the mandatory constructor and write methods for a given proxified class.
+   * @param <T> the proxified class
+   */
+  protected static class Type<T extends Serializable> {
+    private final Constructor<T>[] ctors;
+    private final Method[] writes;
+    private transient int slot = 255;
+
+    /**
+     * Creates a new instance of type.
+     * @param clazz the proxified class
+     */
+    public Type(Class<T> clazz) {
+        ctors = typeConstructors(clazz);
+        writes = typeWrites(clazz);
+    }
+
+    /**
+     * The slot number if the class is registered.
+     * @return the slot number, 255 means not-registered
+     */
+    public int getSlot() {
+      return slot;
+    }
+
+    /**
+     * @return the target class
+     */
+    public String getTargetName() {
+      // there is always at least one ctor
+      return ctors[0].getDeclaringClass().getName();
+    }
+
+    /**
+     * Compare parameter signatures of executables.
+     * @param lhs left-hand side
+     * @param rhs right-hand side
+     * @return 0 if equal, +/- 1 if left &lt;/&gt; than right
+     */
+    private static int compareSignatures(Executable lhs, Executable rhs) {
+      return compareSignatures(lhs.getParameterTypes(), rhs.getParameterTypes());
+    }
+
+    /**
+     * Compare executables parameter signatures.
+     * @param lhs left-hand side executable
+     * @param rhs right-hand side executable
+     * @return 0 if equal, +/- 1 if left &lt;/&gt; than right
+     */
+    private static int compareSignatures(Class<?>[] lhs, Class<?>[] rhs) {
+      if (lhs.length < rhs.length) {
+        return -1;
+      }
+      if (lhs.length > rhs.length) {
+        return 1;
+      }
+      int cmp = 0;
+      // lhs.length == rhs.length
+      final int length = lhs.length;
+      for (int p = 0; p < length; ++p) {
+        Class<?> actual = lhs[p];
+        Class<?> formal = rhs[p];
+        if (formal != null && actual != null && !formal.isAssignableFrom(actual)) {
+          // if formal parameter is primitive and actual argument is compatible
+          int dist;
+          if (formal.isPrimitive() && (dist = CONVERTIBLES.get(formal).indexOf(actual)) >= 0) {
+            cmp +=  dist;
+            continue;
+          }
+          dist = formal.getName().compareTo(actual.getName());
+          if (dist != 0) {
+            return cmp * (length - p);

Review Comment:
   nit: If there is a case `cmp * (length - p)` = 0 though the args mismatch?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187433359


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyMap.java:
##########
@@ -0,0 +1,466 @@
+/*
+ * 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.properties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * A property map pertaining to a given object type (cluster, database, table).
+ * <p>
+ *   Maps follow a copy-on-write scheme gated by a dirty flag (avoid copy of a dirty map). This allows
+ *   sharing their content (the inner properties map) with guaranteed isolation and thread safety.
+ * </p>
+ */
+public class PropertyMap implements Serializable {

Review Comment:
   The idea behind the serialization proxy is that of a convention; the 'contract' does not need to be made explicit through inheritance. Adding an abstract class for one use case that is already provably following a convention does not bring any clarity imho.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187336490


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -0,0 +1,307 @@
+/* * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonSyntaxException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyManager;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.Source;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.Part;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * The property  cli servlet.
+ */
+public class PropertyServlet extends HttpServlet {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyServlet.class);
+  /** The object store. */
+  private final RawStore objectStore;
+  /** The security. */
+  private final ServletSecurity security;
+
+  PropertyServlet(Configuration configuration, RawStore store) {

Review Comment:
   This the servlet allowing the properties features to be accessed through https.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1552041091

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [112 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1204596555


##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/properties/HMSServletTest.java:
##########
@@ -0,0 +1,278 @@
+/* * 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.properties;
+
+import com.google.gson.Gson;
+import com.nimbusds.jose.JWSAlgorithm;
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.JWSSigner;
+import com.nimbusds.jose.crypto.RSASSASigner;
+import com.nimbusds.jose.jwk.RSAKey;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.SignedJWT;
+
+import com.github.tomakehurst.wiremock.junit.WireMockRule;
+import org.apache.commons.httpclient.methods.PostMethod;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.PropertyServlet;
+import org.apache.hadoop.hive.metastore.ServletSecurity;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpMethod;
+import org.apache.commons.httpclient.NameValuePair;
+import org.apache.commons.httpclient.methods.DeleteMethod;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.commons.httpclient.methods.PutMethod;
+import org.apache.commons.httpclient.methods.StringRequestEntity;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.eclipse.jetty.server.Server;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletResponse;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.get;
+import static com.github.tomakehurst.wiremock.client.WireMock.ok;
+
+public class HMSServletTest extends HMSTestBase {
+
+  private static String baseDir = System.getProperty("basedir");
+  private static final File jwtAuthorizedKeyFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-authorized-key.json");
+  private static final File jwtUnauthorizedKeyFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-unauthorized-key.json");
+  private static final File jwtVerificationJWKSFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-verification-jwks.json");
+
+  public static final String USER_1 = "USER_1";
+
+  private static final int MOCK_JWKS_SERVER_PORT = 8089;
+  @ClassRule
+  public static final WireMockRule MOCK_JWKS_SERVER = new WireMockRule(MOCK_JWKS_SERVER_PORT);
+  // the url part
+  private static final String CLI = "hmscli";
+  Server servletServer = null;
+  int sport = -1;
+
+
+  @Override protected int createServer(Configuration conf) throws Exception {
+    // need store before server for servlet
+    if (objectStore == null) {
+      MetastoreConf.setVar(conf, MetastoreConf.ConfVars.PROPERTIES_SERVLET_AUTH, "JWT");
+      MetastoreConf.setVar(conf, MetastoreConf.ConfVars.THRIFT_METASTORE_AUTHENTICATION_JWT_JWKS_URL,
+          "http://localhost:" + MOCK_JWKS_SERVER_PORT + "/jwks");
+      MOCK_JWKS_SERVER.stubFor(get("/jwks")
+          .willReturn(ok()
+              .withBody(Files.readAllBytes(jwtVerificationJWKSFile.toPath()))));
+      boolean inited = createStore(conf);
+      LOG.info("MetaStore store initialization " + (inited ? "successful" : "failed"));
+    }
+    if (servletServer == null) {
+      servletServer = PropertyServlet.startServer(conf, CLI, objectStore);
+      if (!servletServer.isStarted()) {
+        Assert.fail("http server did not start");
+      }
+      sport = servletServer.getURI().getPort();
+    }
+    return sport;
+  }
+
+  /**
+   * Stops the server.
+   * @param port the server port
+   */
+  @Override protected void stopServer(int port) throws Exception {
+    if (servletServer != null) {
+      servletServer.stop();
+      servletServer = null;
+      sport = -1;
+    }
+  }
+
+  @Override
+  protected JSonClient createClient(Configuration conf, int sport) throws Exception {
+    URL url = new URL("http://hive@localhost:" + sport + "/" + CLI + "/" + NS);
+    String jwt = generateJWT();
+    return new JSonClient(jwt, url);
+  }
+
+  private String generateJWT()  throws Exception {
+    return generateJWT(USER_1, jwtAuthorizedKeyFile.toPath(), TimeUnit.MINUTES.toMillis(5));
+  }
+
+  private static String generateJWT(String user, Path keyFile, long lifeTimeMillis) throws Exception {
+    RSAKey rsaKeyPair = RSAKey.parse(new String(java.nio.file.Files.readAllBytes(keyFile), StandardCharsets.UTF_8));
+    // Create RSA-signer with the private key
+    JWSSigner signer = new RSASSASigner(rsaKeyPair);
+    JWSHeader header = new JWSHeader
+        .Builder(JWSAlgorithm.RS256)
+        .keyID(rsaKeyPair.getKeyID())
+        .build();
+    Date now = new Date();
+    Date expirationTime = new Date(now.getTime() + lifeTimeMillis);
+    JWTClaimsSet claimsSet = new JWTClaimsSet.Builder()
+        .jwtID(UUID.randomUUID().toString())
+        .issueTime(now)
+        .issuer("auth-server")
+        .subject(user)
+        .expirationTime(expirationTime)
+        .claim("custom-claim-or-payload", "custom-claim-or-payload")
+        .build();
+    SignedJWT signedJWT = new SignedJWT(header, claimsSet);
+    // Compute the RSA signature
+    signedJWT.sign(signer);
+    return signedJWT.serialize();
+  }
+
+
+  /**
+   * A property client that uses http as transport.
+   */
+  public static class JSonClient implements PropertyClient {
+    private final URL url;
+    private String jwt = null;
+    JSonClient(String token, URL url) {
+      this.jwt = token;
+      this.url = url;
+    }
+
+    public boolean setProperties(Map<String, String> properties) {
+      try {
+        clientCall(jwt, url, "PUT", properties);
+        return true;
+      } catch(IOException xio) {
+        return false;
+      }
+    }
+
+    public Map<String, Map<String, String>> getProperties(String mapPrefix, String mapPredicate, String... selection) throws IOException {
+      Map<String, Object> args = new TreeMap<>();
+      args.put("prefix", mapPrefix);
+      if (mapPredicate != null) {
+        args.put("predicate", mapPredicate);
+      }
+      if (selection != null && selection.length > 0) {
+        args.put("selection", selection);
+      }
+      try {
+        Object result = clientCall(jwt, url, "POST", args);
+        return result instanceof Map? (Map<String, Map<String, String>>) result : null ;
+      } catch(IOException xio) {
+        return null;
+      }
+    }
+  }
+
+  @Test
+  public void testJSONServlet() throws Exception {
+    URL url = new URL("http://hive@localhost:" + sport + "/" + CLI + "/" + NS);
+    Map<String, String> json = Collections.singletonMap("method", "echo");
+    String jwt = generateJWT();
+    Object response = clientCall(jwt, url, "POST", json);
+    Assert.assertNotNull(response);
+    Assert.assertEquals(json, response);
+
+    response = clientCall(null, url, "POST", json);
+    Assert.assertNull(response);
+  }
+
+  @Test
+  public void testProperties1() throws Exception {
+      runOtherProperties1(client);
+
+  }
+
+  @Test
+  public void testProperties0() throws Exception {
+      runOtherProperties0(client);
+  }
+
+  @Test
+  public void testPropertiesOtherClient() throws Exception {
+    HttpClient client = new HttpClient();
+    HttpMethod method = new PostMethod("http://hive@localhost:" + sport + "/" + CLI + "/" + NS);
+
+    String jwt = generateJWT();
+    method.addRequestHeader("Authorization","Bearer " + jwt);
+    method.addRequestHeader("Content-Type", "application/json");
+    method.addRequestHeader("Accept", "application/json");
+
+    String msgBody = "{\"method\":\"echo\"}";

Review Comment:
   Added.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171289645


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -7535,6 +7523,38 @@ public List<Partition> get_partitions_by_names(final String dbName, final String
     return ret;
   }
 
+  /**
+   * Creates an instance of property manager based on the (declared) namespace.
+   * @param ns the namespace
+   * @return the manager instance
+   * @throws TException
+   */
+  private PropertyManager getPropertyManager(String ns) throws TException {
+    PropertyStore propertyStore = getMS().getPropertyStore();
+    PropertyManager mgr = PropertyManager.create(ns, propertyStore);
+    return mgr;
+  }
+  @Override
+  public PropertyGetResponse get_properties(PropertyGetRequest req) throws TException {
+    PropertyManager mgr = getPropertyManager(req.getNameSpace());
+    Map<String, PropertyMap> selected = mgr.selectProperties(req.getMapPrefix(), req.getMapPredicate(), req.getMapSelection());

Review Comment:
   nit: seems like we can make `selectProperties` static in `PropertyManager`, other methods as well



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1209911873


##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/properties/HMSServletTest.java:
##########
@@ -63,21 +67,21 @@
 
 public class HMSServletTest extends HMSTestBase {

Review Comment:
   They do use different server/client implementations (direct/thrift/servlet + URL/HttpClient) to perform the same calls.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1569302037

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [92 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174273842


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -2769,6 +2802,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   GetPartitionsByNamesResult get_partitions_by_names_req(1:GetPartitionsByNamesRequest req)
                         throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
+    // retrieve properties
+    PropertyGetResponse get_properties(1:PropertyGetRequest req);

Review Comment:
   All HiveMetaStoreClient (thrift related) methods throw TException.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174223446


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/SoftCache.java:
##########
@@ -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.hadoop.hive.metastore.properties;
+
+import java.lang.ref.SoftReference;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * A soft referenced cache.
+ * <p>
+ * The actual cache is held through a soft reference, allowing it to be GCed under memory pressure.</p>
+ * <p>
+ * This class is <em>not</em> thread-safe.</p>
+ * @param <K> the cache key entry type
+ * @param <V> the cache key value type
+ */
+public class SoftCache<K, V> {
+    /** The default cache capacity. */
+    private static final int CACHE_CAPACITY = 64;
+    /** The default cache load factor. */
+    private static final float LOAD_FACTOR = 0.75f;

Review Comment:
   Should we make this soft cache configurable? Do you see any downside by doing so?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174279570


##########
standalone-metastore/metastore-server/src/test/resources/org/apache/hadoop/hive/metastore/properties/pol0.json:
##########
@@ -0,0 +1,24 @@
+{

Review Comment:
   Testing content.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1508959914

   Note to reviewers:
   most of the code is in hive-standalone-metastore-common under org.apache.hadoop.hive.metastore.properties; the JDO class is JdoPropertyStore in hive-standalone-metastore-server. Test reside in both projects under org.apache.hadoop.hive.metastore.properties.


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171695068


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -5192,4 +5192,25 @@ public List<WriteEventInfo> getAllWriteEventInfo(GetAllWriteEventInfoRequest req
   public AbortCompactResponse abortCompactions(AbortCompactionRequest request) throws TException{
     return client.abort_Compactions(request);
   }
+
+  @Override
+  public boolean setProperties(String nameSpace, Map<String, String> properties) throws TException {
+    PropertySetRequest psr = new PropertySetRequest();
+    psr.setNameSpace(nameSpace);
+    psr.setPropertyMap(properties);
+    return client.set_properties(psr);
+  }
+
+  @Override
+  public Map<String, Map<String, String>> getProperties(String nameSpace, String mapPrefix, String mapPredicate, String... selection) throws TException {
+    PropertyGetRequest request = new PropertyGetRequest();
+    request.setNameSpace(nameSpace);
+    request.setMapPrefix(mapPrefix);
+    request.setMapPredicate(mapPredicate);
+    if (selection != null && selection.length > 0) {
+      request.setMapSelection(Arrays.asList(selection));
+    }
+    PropertyGetResponse response = client.get_properties(request);

Review Comment:
   should we do this instead `return client.get_properties(request).getProperties()`?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171892570


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/HMSPropertyManager.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.properties;
+
+import org.apache.hadoop.hive.metastore.api.MaintenanceOpStatus;
+import org.apache.hadoop.hive.metastore.api.MaintenanceOpType;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+
+/**
+ * A property manager tailored for the HiveMetaStore.
+ * It describes properties for cluster, database and table based on declared schemas.
+ * A property is of the form:
+ * <ul>
+ *   <li>name : when it refers to a cluster property named 'name'</li>
+ *   <li>db.name : when it refers to a database property named 'name' for the database 'db'</li>
+ *   <li>db.table.name : when it refers to a table property named 'name' for the table 'table' in the database 'db</li>
+ * </ul>
+ */
+public class HMSPropertyManager extends PropertyManager {
+  private static final  String CLUSTER_PREFIX = "cluster";
+  private static final String DATABASE_PREFIX = "database";
+  private static final String TABLE_PREFIX = "table";
+  /* Declare HMS. */
+  static {
+    PropertyManager.declare("hms", HMSPropertyManager.class);
+  }
+
+  /**
+   * Maintenance Operation Type.
+   */
+  public static final PropertyType<MaintenanceOpType> MAINTENANCE_OPERATION = new PropertyType<MaintenanceOpType>("MaintenanceOperation"){

Review Comment:
   nit: space between `){`



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171701610


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java:
##########
@@ -4393,4 +4393,27 @@ ReplicationMetricList getReplicationMetrics(GetReplicationMetricsRequest
 
   AbortCompactResponse abortCompactions(AbortCompactionRequest request) throws TException;
 
+  /**
+   * Sets properties.
+   * @param nameSpace the property store namespace
+   * @param properties a map keyed by property path mapped to property values
+   * @return true if successful, false otherwise
+   * @throws TException
+   */
+  default boolean setProperties(String nameSpace, Map<String, String> properties) throws TException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Gets properties.
+   * @param nameSpace the property store namespace.
+   * @param mapPrefix the map prefix (ala starts-with) to select maps
+   * @param mapPredicate predicate expression on properties to further reduce the selected maps
+   * @param selection the list of properties to return, null for all
+   * @return a map keyed by property map path to maps keyed by property name mapped to property values
+   * @throws TException
+   */
+  default Map<String, Map<String, String>> getProperties(String nameSpace, String mapPrefix, String mapPredicate, String... selection) throws TException {
+    throw new UnsupportedOperationException();

Review Comment:
   Same as above.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1522815351

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [103 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1177644213


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MMetastoreDBProperties.java:
##########
@@ -21,6 +21,8 @@ public class MMetastoreDBProperties {
   private String propertyKey;
   private String propertyValue;
   private String description;
+  private byte[] propertyContent;

Review Comment:
   Adding the schema updating scripts.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1549503690

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [110 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1557418728

   Recreated from cleaner base.


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1571250050

   The failed test: TestTokenStoreDelegationTokenSecretManager may be caused by the added dependency: commons-httpclient in metastore-server


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1209913602


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Digester.java:
##########
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hive.metastore.properties;
 
+import org.jetbrains.annotations.NotNull;

Review Comment:
   Slipped through the cracks, thanks.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1569871464

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [92 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174278092


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property mamanger class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized(NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor != null) {
+      try {
+        return ctor.newInstance(namespace, store);
+      } catch(Exception xany) {
+        LOGGER.error("property manager creation failed "+ namespace, xany);
+      }
+    } else {
+      LOGGER.error("no such property manager namespace is declared " + namespace);

Review Comment:
   An error is not silent :-) ; it also should only occur during development of a new PropertyManager since it means the constructor failed. I'm trying to avoid throwing an exception that could crash the server or avoid it to start.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187278590


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,629 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.JexlScript;
+import org.apache.commons.jexl3.ObjectContext;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ * <p>The manager class needs to be registered with its namespace as key</p>
+ * <p>
+ *   Since a collection of properties are stored in a map, to avoid hitting the persistence store for each update
+ *   - which would mean rewriting the map multiple times - the manager keeps track of dirty maps whilst
+ *   serving as transaction manager. This way, when importing multiple properties targeting different elements (think
+ *   setting properties for different tables), each impacted map is only rewritten
+ *   once by the persistence layer during commit. This also allows multiple calls to participate to one transactions.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected final Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property manager class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   * @throws MetaException if the manager creation fails
+   * @throws NoSuchObjectException if the store is null or no constructor was declared
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) throws MetaException, NoSuchObjectException {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized (NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor == null) {
+      throw new NoSuchObjectException("no PropertyManager namespace is declared, namespace " + namespace);
+    }
+    if (store == null) {
+      throw new NoSuchObjectException("no PropertyStore exists " + namespace);
+    }
+    try {
+      return ctor.newInstance(namespace, store);
+    } catch (Exception xany) {
+      LOGGER.error("PropertyManager creation failed " + namespace, xany);
+      throw new MetaException("PropertyManager creation failed, namespace " + namespace);
+    }
+  }
+
+  /**
+   * JEXL adapter.
+   * <p>public for introspection.</p>
+   */
+  public static class MapWrapper implements JexlContext {
+    PropertyMap map;
+    MapWrapper(PropertyMap map) {
+      this.map = map;
+    }
+
+    public Object get(String p) {
+      return map.getPropertyValue(p);
+    }
+
+    @Override
+    public void set(String name, Object value) {
+      map.putProperty(name, value);
+    }
+
+    @Override
+    public boolean has(String name) {
+      return map.getTypeOf(name) != null;
+    }
+  }
+
+  /**
+   * Creates a manager instance.
+   * @param store the store instance which must use an appropriate property map factory (probably use createMap).
+   */
+  protected PropertyManager(String ns, PropertyStore store) {
+    this.namespace = ns;
+    this.store = store;
+  }
+
+  /**
+   * Saves all pending updates to store.
+   */
+  public void commit() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      if (!dirtyMaps.isEmpty()) {
+        store.saveProperties(dirtyMaps.entrySet().iterator());
+        dirtyMaps.clear();
+      }
+    }
+  }
+
+  /**
+   * Forget all pending updates.
+   */
+  public void rollback() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      dirtyMaps.clear();
+    }
+  }
+
+  /**
+   * Imports a set of default values into this store&quot;s schema.
+   * The properties should be of the form schema_name.property_name=value.
+   * Note that this implies the manager has at least one known property map schema.
+   * @param importsp the properties
+   */
+  public void importDefaultValues(Properties importsp) {
+    importsp.forEach((k, v)->{
+      String importName = k.toString();
+      final int dotPosition = importName.indexOf(".");
+      if (dotPosition > 0) {
+        String schemaName = importName.substring(0, dotPosition);
+        PropertySchema schema = getSchema(schemaName);
+        if (schema != null) {
+          String propertyName = importName.substring(dotPosition + 1);
+          schema.setDefaultValue(propertyName, v);
+        }
+      }
+    });
+  }
+
+  /**
+   * Imports a set of property values.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param map the properties key=value
+   */
+  public void setProperties(Properties map) {
+    map.forEach((k, v)-> setProperty(k.toString(), v));
+  }
+
+  /**
+   * Injects a set of properties.
+   * If the value is null, the property is removed.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param map the map of properties to inject.
+   */
+  public void setProperties(Map<String, ?> map) {
+    map.forEach(this::setProperty);
+  }
+
+  /**
+   * Sets a property value.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param key the property key
+   * @param value the property value or null to unset
+   */
+  public void setProperty(String key, Object value) {
+    setProperty(splitKey(key), value);
+  }
+
+  /**
+   * Runs a JEXL script using this manager as context.
+   * @param src the script source
+   * @return the script result
+   * @throws PropertyException if any error occurs in JEXL
+   */
+  public Object runScript(String src) throws PropertyException {
+    try {
+      JexlScript script = JEXL.createScript(src);
+      ObjectContext<PropertyManager> context = new ObjectContext<>(JEXL, this);
+      return script.execute(context);
+    } catch(JexlException je) {
+      throw new PropertyException("script failed", je);
+    }
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or null if not assigned
+   */
+  public Object getProperty(String key) {
+    return getProperty(splitKey(key));
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or the schema default value if not assigned
+   */
+  public Object getPropertyValue(String key) {
+    return getPropertyValue(splitKey(key));
+  }
+
+  /**
+   * Splits a property key into its fragments.
+   * @param key the property key
+   * @return the key fragments
+   */
+  protected String[] splitKey(String key) {
+    String[] splits = key.split("(?<!\\\\)\\.");
+    if (splits.length < 1) {
+      splits = new String[]{key};
+    }
+    return splits;
+  }
+
+  /**
+   * Gets a schema by name.
+   * <p>Only used by {@link #importDefaultValues(Properties)}</p>
+   * @param name schema name
+   * @return the schema instance, null if no such schema is known
+   */
+  public PropertySchema getSchema(String name) {
+    return null;
+  }
+
+  /**
+   * Determines the schema from the property key fragments.
+   * @param keys the key fragments
+   * @return the schema, {@link PropertySchema#NONE} if no such schema is known
+   */
+  protected PropertySchema schemaOf(String[] keys) {
+    return PropertySchema.NONE;
+  }
+
+  /**
+   * @param keys property key fragments
+   * @return number of fragments composing the map name in the fragments array
+   */
+  protected int getMapNameLength(String[] keys) {
+    return keys.length - 1;
+  }
+
+  /**
+   * Compose a property map key from a property map name.
+   * @param name the property map name, may be null or empty
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String name) {
+    StringBuilder strb = new StringBuilder(namespace);
+    if (name != null && !name.isEmpty()){
+      strb.append('.');
+      strb.append(name);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Extract a property map name from a property map key.
+   * @param key property map key
+   * @return the property map name
+   */
+  protected String mapName(String key) {
+    int dot = key.indexOf('.');
+    return dot > 0? key.substring(dot + 1) : key;
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the key fragments
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String[] keys) {
+    return mapKey(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the property key fragments
+   * @param maxkl the maximum number of fragments in the map key
+   * @return the property key used by the store
+   */
+  protected String mapKey(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    // shortest map key is namespace
+    StringBuilder strb = new StringBuilder(namespace);
+    for(int k = 0; k < Math.min(maxkl, keys.length - 1); ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @return the property name
+   */
+  protected String propertyName(String[] keys) {
+    return propertyName(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @param maxkl the maximum number of fragments in the map name
+   * @return the property name
+   */
+  protected String propertyName(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    if (keys.length <= maxkl) {
+      return keys[keys.length - 1];
+    }
+    StringBuilder strb = new StringBuilder(keys[maxkl]);
+    for(int k = maxkl + 1; k < keys.length; ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or null if none was assigned
+   */
+  public Object getProperty(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    if (map == null) {
+      map = store.fetchProperties(mapKey, null);
+    }
+    if (map != null) {
+      return map.getProperty(propertyName(keys));
+    }
+    return null;
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or the default schema value if not assigned
+   */
+  public Object getPropertyValue(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    PropertySchema schema = schemaOf(keys);
+    if (map == null) {
+      map = store.fetchProperties(mapKey, s->schema);
+    }
+    String propertyName = propertyName(keys);
+    if (map != null) {
+      return map.getPropertyValue(propertyName);
+    }
+    if (schema != null) {
+      return schema.getDefaultValue(propertyName);
+    }
+    return null;
+  }
+
+  /**
+   * Drops a property map.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param mapName the map name
+   * @return true if the properties may exist, false if they did nots
+   */
+  public boolean dropProperties(String mapName) {
+    final String mapKey = mapKey(mapName);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null && Objects.equals(PropertyMap.DROPPED, dirtyMap.getDigest())) {
+      map = dirtyMap;
+    } else {
+      // is it stored ?
+      UUID digest = store.fetchDigest(mapKey);
+      // not stored nor cached, nothing to do
+      if (digest == null) {
+        return false;
+      }
+      map = new PropertyMap(store, schemaOf(splitKey(mapName + ".*")), PropertyMap.DROPPED);
+      synchronized (dirtyMaps) {
+        dirtyMaps.put(mapName, map);
+      }
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapName, map);
+    }
+    return false;
+  }
+
+  /**
+   * Sets a property value.
+   * @param keys the key fragments
+   * @param value the new value or null if mapping should be removed
+   */
+  protected void setProperty(String[] keys, Object value) {
+    // find schema from key (length)
+    PropertySchema schema = schemaOf(keys);
+    String mapKey = mapKey(keys);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null) {
+      map = dirtyMap;
+    } else {
+      // is is stored ?
+      map = store.fetchProperties(mapKey, s->schema);
+      if (map == null) {
+        // remove a value from a non persisted map, noop
+        if (value == null) {
+          return;
+        }
+        map = new PropertyMap(store, schema);
+      }
+    }
+    // map is not null
+    String propertyName = propertyName(keys);
+    if (value != null) {
+      map.putProperty(propertyName, value);
+    } else {
+      map.removeProperty(propertyName);
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapKey, map);
+    }
+  }
+
+  /**
+   * Selects a set of properties.
+   * @param namePrefix the map name prefix
+   * @param predicateStr the condition selecting maps
+   * @param projectStr the projection property names or script
+   * @return the map of property maps keyed by their name
+   */
+  public Map<String, PropertyMap> selectProperties(String namePrefix, String predicateStr, String... projectStr) {
+    return selectProperties(namePrefix, predicateStr,
+        projectStr == null
+            ? Collections.emptyList()
+            : Arrays.asList(projectStr));
+  }
+
+  /**
+   * Selects a set of properties.
+   * @param namePrefix the map name prefix
+   * @param selector the selector/transformer function
+   * @return the map of property maps keyed by their name
+   */
+  public Map<String, PropertyMap> selectProperties(String namePrefix, Function<PropertyMap, PropertyMap> selector) {
+    final String mapKey = mapKey(namePrefix);
+    final Map<String, PropertyMap> selected = store.selectProperties(mapKey,null, k->schemaOf(splitKey(k)) );
+    final Map<String, PropertyMap> maps = new TreeMap<>();
+    final Function<PropertyMap, PropertyMap> transform = selector == null? Function.identity() : selector;
+    selected.forEach((k, p) -> {
+      final PropertyMap dirtyMap = dirtyMaps.get(k);

Review Comment:
   nit: If I understand correctly, the `dirtyMaps` will always be empty for `selectProperties` method



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1562028809

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [100 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174228581


##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/properties/HMSPropertyStoreRemoteTest.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.jexl3.JxltEngine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HMSHandler;
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.JdoPropertyStore;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.ObjectStore;
+import org.apache.hadoop.hive.metastore.TestObjectStore;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MaintenanceOpStatus;
+import org.apache.hadoop.hive.metastore.api.MaintenanceOpType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.hive.metastore.properties.HMSPropertyManager.JEXL;
+import static org.apache.hadoop.hive.metastore.properties.HMSPropertyManager.MAINTENANCE_OPERATION;
+import static org.apache.hadoop.hive.metastore.properties.HMSPropertyManager.MAINTENANCE_STATUS;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.BOOLEAN;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.DATETIME;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.DOUBLE;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.INTEGER;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.JSON;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.STRING;
+
+public class HMSPropertyStoreRemoteTest extends HMSPropertyStoreTest {
+  //private static final String NS = "hms";
+  protected HiveMetaStoreClient client;
+  private boolean isServerStarted = false;
+  protected int port;
+
+  boolean createStore(Configuration conf, Warehouse wh) {
+    try {
+      MetaStoreTestUtils.setConfForStandloneMode(conf);
+      objectStore = new ObjectStore();
+      objectStore.setConf(conf);
+      //TestObjectStore.dropAllStoreObjects(objectStore);
+      HMSHandler.createDefaultCatalog(objectStore, wh);
+      // configure object store
+      objectStore.createDatabase(new DatabaseBuilder()
+          .setCatalogName("hive")
+          .setName(DB1)
+          .setDescription("description")
+          .setLocation("locationurl")
+          .build(conf));
+    } catch(InvalidObjectException | MetaException | InvalidOperationException xmeta) {
+      throw new PropertyException("unable to initialize server", xmeta);
+    }
+    return true;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
+    // Events that get cleaned happen in batches of 1 to exercise batching code
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.EVENT_CLEAN_MAX_EVENTS, 1L);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    if (isServerStarted) {
+      Assert.assertNotNull("Unable to connect to the MetaStore server", client);
+      return;
+    }
+    port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    System.out.println("Starting MetaStore Server on port " + port);
+    isServerStarted = true;
+
+    Warehouse wh = new Warehouse(conf);
+    boolean inited = createStore(conf, wh);
+    LOG.info("MetaStore Thrift Server test initialization " + (inited? "successful":"failed"));
+    // This is default case with setugi off for both client and server
+    client = createClient();
+  }
+
+
+  protected HiveMetaStoreClient createClient() throws Exception {
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.EXECUTE_SET_UGI, false);
+    HiveMetaStoreClient client = new HiveMetaStoreClient(conf);
+    return client;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    MetaStoreTestUtils.close(port);
+    super.tearDown();
+  }
+
+  @Test
+  public void testHMSProperties() throws Exception {

Review Comment:
   I think we can just setup the config to use remote HMS and then run the tests from HMSPropertyStoreTest class. We currently do that for few HMS unit tests.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174224277


##########
standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/properties/PocExternal.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.properties;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+
+/**
+ * A Proof Of Concept about externalizable.
+ */
+public class PocExternal implements Serializable {

Review Comment:
   Do we need this POC in the hive repo?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174270529


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/JdoPropertyStore.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.commons.jexl3.JexlException;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertySchema;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * Implementation of the property store delegating persistence to a (jdo) raw store.
+ */
+public class JdoPropertyStore extends PropertyStore {
+  /** The jdo objects store. */
+  private final RawStore objectStore;
+
+  /**
+   * Basic ctor.
+   * @param store the object store
+   */
+  public JdoPropertyStore(RawStore store) {
+    this.objectStore = store;
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.getProperties(mapKey, getPropertyMapFunction(null, getSchema));

Review Comment:
   The logic is using the persistence manager of the object store; trying to respect the encapsulation...



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174259729


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MMetastoreDBProperties.java:
##########
@@ -21,6 +21,8 @@ public class MMetastoreDBProperties {
   private String propertyKey;
   private String propertyValue;
   private String description;
+  private byte[] propertyContent;

Review Comment:
   We actually do add a column which is a blob; the update scripts are lacking.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171892924


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/HMSPropertyManager.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.properties;
+
+import org.apache.hadoop.hive.metastore.api.MaintenanceOpStatus;
+import org.apache.hadoop.hive.metastore.api.MaintenanceOpType;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+
+/**
+ * A property manager tailored for the HiveMetaStore.
+ * It describes properties for cluster, database and table based on declared schemas.
+ * A property is of the form:
+ * <ul>
+ *   <li>name : when it refers to a cluster property named 'name'</li>
+ *   <li>db.name : when it refers to a database property named 'name' for the database 'db'</li>
+ *   <li>db.table.name : when it refers to a table property named 'name' for the table 'table' in the database 'db</li>
+ * </ul>
+ */
+public class HMSPropertyManager extends PropertyManager {
+  private static final  String CLUSTER_PREFIX = "cluster";
+  private static final String DATABASE_PREFIX = "database";
+  private static final String TABLE_PREFIX = "table";
+  /* Declare HMS. */
+  static {
+    PropertyManager.declare("hms", HMSPropertyManager.class);
+  }
+
+  /**
+   * Maintenance Operation Type.
+   */
+  public static final PropertyType<MaintenanceOpType> MAINTENANCE_OPERATION = new PropertyType<MaintenanceOpType>("MaintenanceOperation"){
+    @Override public MaintenanceOpType cast(Object value) {
+      if (value instanceof MaintenanceOpType) {
+        return (MaintenanceOpType) value;
+      }
+      if (value == null) {
+        return null;
+      }
+      if(value instanceof Number) {
+        return MaintenanceOpType.findByValue(((Number) value).intValue());
+      }
+      return parse(value.toString());
+    }
+    @Override public MaintenanceOpType parse(String str) {
+      if (str == null) {
+        return null;
+      }
+      return MaintenanceOpType.valueOf(str.toUpperCase());
+    }
+
+    @Override public String format(Object value) {
+      if (value instanceof MaintenanceOpType) {
+        return value.toString();
+      }
+      return null;
+    }
+  };
+
+  /**
+   * Maintenance Operation Status.
+   */
+  public static final PropertyType<MaintenanceOpStatus> MAINTENANCE_STATUS = new PropertyType<MaintenanceOpStatus>("MaintenanceStatus"){

Review Comment:
   nit: same as above



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174259080


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Serializer.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.properties;
+
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.file.Files;
+
+/**
+ * A helper class to read/write objects through the SerializationProxy.
+ *
+ * @param <T>
+ */
+public class Serializer<T extends Serializable> {
+  private static final Logger LOGGER = SerializationProxy.LOGGER;
+

Review Comment:
   I'll look into it; the Serializer/SerializationProxy allows a very clean control and very compact representation. I had a look at SerializationUtilities : they are in ql, depend on Kryo and the feature set is a bit different at first glance. I'd rather stick with the proxy that does not rely on anything but a convention on the constructor/write method.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1525126670

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [104 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175399200


##########
standalone-metastore/metastore-server/src/test/resources/org/apache/hadoop/hive/metastore/properties/pol0.json:
##########
@@ -0,0 +1,24 @@
+{

Review Comment:
   In HMSTestBase.createProperties0(...).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171633934


##########
standalone-metastore/metastore-common/pom.xml:
##########
@@ -25,6 +25,11 @@
     <standalone.metastore.path.to.root>..</standalone.metastore.path.to.root>
   </properties>
   <dependencies>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-jexl3</artifactId>
+      <version>3.3</version>

Review Comment:
   Can we replace this with place holder and define the actual version standalone-metastore/pom.xml?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175575362


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -2769,6 +2802,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   GetPartitionsByNamesResult get_partitions_by_names_req(1:GetPartitionsByNamesRequest req)
                         throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
+    // retrieve properties
+    PropertyGetResponse get_properties(1:PropertyGetRequest req);

Review Comment:
   Adding exceptions triggered when creating the manager fail.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175628772


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Digester.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.properties;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Date;
+import java.util.UUID;
+
+/**
+ * Helper class that creates a type 5 uuid.
+ * <p>This is computed from a set of updates using a SHA-1 message digest massaged into an UUID.
+ * see <a href="https://en.wikipedia.org/wiki/Universally_unique_identifier">...</a>
+ */
+public class Digester {
+    /** The Namespace uuid. */
+    private final UUID nsuid;
+    /** The digest used to compute the UUID. */
+    private final MessageDigest md;
+    /** A default namespace based on the class loading time. */
+    private static final UUID TEMP_NS;
+    static {
+        MessageDigest md = createDigest();
+        digest(md, System.currentTimeMillis());
+        TEMP_NS = computeUUID(md);
+    }
+
+    /**
+     * Allows to update the message digest from an object.
+     */
+    private static class TagOutputStream extends OutputStream {
+        /** The digest to update. */
+        private final MessageDigest md;
+
+        /**
+         * Sole ctor.
+         * @param md the message digester
+         */
+        TagOutputStream(MessageDigest md) {
+            this.md = md;
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            md.update((byte) b);
+        }
+
+        @Override
+        public void write(byte[] b, int off, int len) throws IOException {
+            md.update(b, off, len);
+        }
+    }
+
+    /**
+     * @return a SHA-1 message digest
+     */
+    private static MessageDigest createDigest() {
+        MessageDigest md;
+        try {
+            md = MessageDigest.getInstance("SHA-1");
+        } catch (NoSuchAlgorithmException nsae) {
+            throw new InternalError("SHA not supported");
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with a uuid.
+     * @param md the digest to update
+     * @param uid the uuid
+     */
+    private static MessageDigest digest(MessageDigest md, UUID uid) {
+        if (uid != null) {
+            long msb = uid.getMostSignificantBits();
+            digest(md, msb);
+            long lsb = uid.getLeastSignificantBits();
+            digest(md, lsb);
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with an input stream.
+     * @param md the digest to update
+     * @param input the input to consume
+     * @throws IllegalStateException if an io exception occurs
+     */
+    private static void digest(MessageDigest md, InputStream input) {
+        try (OutputStream out = new TagOutputStream(md)) {
+            byte[] buffer = new byte[1024];
+            int read;
+            while ((read = input.read(buffer, 0, 1024)) >= 0) {
+                out.write(buffer, 0, read);
+            }
+        } catch(IOException xio) {
+            throw new IllegalStateException(xio);
+        }
+    }
+
+
+    /**
+     * Updates a digest with a long.
+     * @param md the digest to update
+     * @param l8 the long
+     */
+    private static void digest(MessageDigest md, long l8) {
+        md.update((byte) (l8 & 0xff));
+        md.update((byte) (l8 >> 8));
+        md.update((byte) (l8 >> 16));
+        md.update((byte) (l8 >> 24));
+        md.update((byte) (l8 >> 32));
+        md.update((byte) (l8 >> 40));
+        md.update((byte) (l8 >> 48));
+        md.update((byte) (l8 >> 56));
+    }
+
+    /**
+     * Updates a digest with an object.
+     * @param md the digest to update
+     * @param obj the object
+     */
+    private static void digest(MessageDigest md, Object obj) {
+        if (obj == null) {
+            return;
+        }
+        try (ObjectOutput out = new ObjectOutputStream(new TagOutputStream(md))) {
+            out.writeObject(obj);
+        } catch (IOException ex) {
+            // ignore close exception
+        }
+        // ignore close exception
+    }
+
+    /**
+     * Computes the uuid.
+     * @param md the message digest used to compute the hash
+     * @return the eTag as a type 5 uuid
+     */
+    private static UUID computeUUID(MessageDigest md) {
+        byte[] sha1Bytes = md.digest();
+        sha1Bytes[6] &= 0x0f;  /* clear version        */
+        sha1Bytes[6] |= 0x50;  /* set to version 5     */
+        sha1Bytes[8] &= 0x3f;  /* clear variant        */
+        sha1Bytes[8] |= 0x80;  /* set to IETF variant  */
+
+        // SHA generates 160 bytes; truncate to 128
+        long msb = 0;
+        //assert data.length == 16 || data.length == 20;
+        for (int i = 0; i < 8; i++) {
+            msb = (msb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        long lsb = 0;
+        for (int i = 8; i < 16; i++) {
+            lsb = (lsb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        return new UUID(msb, lsb);
+    }
+
+    /**
+     * A marker interface for objects that can be digested.
+     */
+    public interface Digestible {
+        /**
+         * Updates a digest with this variable.
+         * @param digester the digester to update
+         * @return true if this digestible actually contributed to the digest
+         */
+        boolean digest(Digester digester);
+    }
+
+    /**
+     * A type 5 uuid is namespace + sha1; namespace in our case is a uuid.
+     * Two instances of digesters built with the same namespace will produce the same UUIDs from the
+     * same inputs.
+     * @param namespace the uuid namespace
+     */
+    public Digester(UUID namespace) {
+        nsuid = namespace == null? TEMP_NS : namespace;
+        md = createDigest();
+        // inject namespace
+        digest(md, nsuid);
+    }
+
+    /**
+     * A copy ctor base.
+     * @param lnsuid the namespace uid
+     * @param lmd the message digest
+     */
+    private Digester(UUID lnsuid, MessageDigest lmd)  {
+        this.nsuid = lnsuid;
+        this.md = lmd;
+    }
+
+    /**
+     * Default ctor.
+     * The created digester uses the class loading time as seed for its namespace; this means 2 instances of digester
+     * built in different JVM instances will *NOT* produce the same UUIDs for the same input. Typical use is in
+     * a non-persistent scenario, to verify an instance of an object has not been modified by checking
+     * its digested UUID remained the same.
+     * To get stable UUID computation across time and space in Digester usable in persistent scenario,
+     * you *NEED* to use a namespace-based digester using {@link Digester(UUID)}, uuid that is easily created
+     * using {@link UUID#nameUUIDFromBytes(byte[])} from any name/uri you might desire.
+     */
+    public Digester() {
+        this(null);
+    }
+
+    /**
+     * @return a clone of this instance
+     */
+    public Digester copy() {
+        try {
+            return new Digester(nsuid, (MessageDigest) md.clone());
+        } catch (CloneNotSupportedException ex) {
+            return null;
+        }
+    }
+
+    /**
+     * Computes the version tag from this digester.
+     * <p>This uses the current message digest state and resets it.
+     * @return the type 5 uuid
+     */
+    public UUID getUUID() {
+        UUID uuid = computeUUID(md);
+        md.reset();
+        digest(nsuid);
+        return uuid;
+    }
+
+    /**
+     * Updates the digest with a boolean.
+     * @param b the boolean
+     * @return this digester
+     */
+    public Digester digest(boolean b) {
+        md.update((byte) (b? 1 : 0));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a char.
+     * @param c the char
+     * @return this digester
+     */
+    public Digester digest(char c) {
+        md.update((byte) (c & 0xff));
+        md.update((byte) (c >> 8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a bytes array.
+     * @param bytes the bytes
+     * @return this digester
+     */
+    public Digester digest(byte[] bytes) {
+        if (bytes != null) {
+            md.update(bytes);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with an integer.
+     * @param i4 the int
+     * @return this digester
+     */
+    public Digester digest(int i4) {
+        md.update((byte) (i4 & 0xff));
+        md.update((byte) (i4 >> 8));
+        md.update((byte) (i4 >> 16));
+        md.update((byte) (i4 >> 24));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a long.
+     * @param l8 the long
+     * @return this digester
+     */
+    public Digester digest(long l8) {
+        digest(md, l8);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a double.
+     * @param f8 the double
+     * @return this digester
+     */
+    public Digester digest(double f8) {
+        digest(md, Double.doubleToRawLongBits(f8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a date.
+     * @param date the date
+     * @return this digester
+     */
+    public Digester digest(Date date) {
+        if (date != null) {
+            digest(md, date.getTime());
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a string.
+     * @param str the string
+     * @return this digester
+     */
+    public Digester digest(String str) {
+        if (str != null) {
+            final int sz = str.length();
+            for(int i = 0; i < sz; ++i) {
+                digest(str.charAt(i));
+            }
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uid the uuid
+     * @return this digester
+     */
+    public Digester digest(UUID uid) {
+        digest(md, uid);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uri the uri
+     * @return this digester
+     */
+    public Digester digest(URI uri) {
+        digest(md, uri.toString());
+        return this;
+    }
+
+    /**
+     * Updates the digest with an object that describes how it digests.
+     * @param digestible the object
+     * @return this digester
+     */
+    public Digester digest(Digestible digestible) {
+        if (digestible != null) {
+            digestible.digest(this);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a stream.
+     * @param input the stream
+     * @return this digester
+     */
+    public Digester digest(InputStream input) {
+        if (input != null) {
+            digest(md, input);
+        }
+        return this;
+    }
+    /**
+     * Updates the digest with any (serializable) object.
+     * @param obj the object
+     * @return this digester
+     */
+    public Digester digest(Object obj) {
+        if (obj instanceof Digestible) {

Review Comment:
   I'm reluctant to try and optimize a method that has not been measured as being a problem. Furthermore, I would expect the 10  'if instanceof'  versus a hash look up to be in the same performance range.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1205101054


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyMap.java:
##########
@@ -0,0 +1,477 @@
+/*
+ * 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.properties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * A property map pertaining to a given object type (cluster, database, table).
+ * <p>
+ *   Maps follow a copy-on-write scheme gated by a dirty flag (avoid copy of a dirty map). This allows
+ *   sharing their content (the inner properties map) with guaranteed isolation and thread safety.
+ * </p>
+ */
+public class PropertyMap implements Serializable {
+  // Vital immutable serialization information.
+  static {
+    SerializationProxy.registerType(0, PropertyMap.class);
+  }
+
+  /**
+   * The logger.
+   */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyMap.class);
+  /**
+   * Serial version.
+   */
+  private static final long serialVersionUID = 202212291759L;
+  /**
+   * The owning store.
+   */
+  protected final transient PropertyStore store;
+  /**
+   * The schema for this map, describes allowed properties and their types.
+   */
+  protected final transient PropertySchema schema;
+  /**
+   * The uuid.
+   */
+  protected transient volatile UUID digest;
+  /**
+   * The properties and their values; the map is cow-once.
+   */
+  protected transient Map<String, Object> properties;
+  /**
+   * Whether this map is dirty which also reflects its copy-on-write state.
+   */
+  protected transient boolean dirty;
+
+  /**
+   * A digest for dropped maps.
+   */
+  static final UUID DROPPED = new Digester().digest(PropertyMap.class.getName()).digest("dropped").getUUID();
+
+  /**
+   * The main ctor.
+   *
+   * @param store  the store this map belongs to
+   * @param schema the schema this map adheres to
+   */
+  PropertyMap(PropertyStore store, PropertySchema schema) {
+    this.store = store;

Review Comment:
   nit: looks like we never use the variable `store`,  can we remove it?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1498982883

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [6 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [92 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187286512


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -0,0 +1,307 @@
+/* * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonSyntaxException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyManager;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.Source;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.Part;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * The property  cli servlet.
+ */
+public class PropertyServlet extends HttpServlet {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyServlet.class);
+  /** The object store. */
+  private final RawStore objectStore;
+  /** The security. */
+  private final ServletSecurity security;
+
+  PropertyServlet(Configuration configuration, RawStore store) {

Review Comment:
   What the purpose of this servlet? if this is for test, can we move it to `test` package?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187276410


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -7536,9 +7524,41 @@ public List<Partition> get_partitions_by_names(final String dbName, final String
     return ret;
   }
 
+  /**
+   * Creates an instance of property manager based on the (declared) namespace.
+   * @param ns the namespace
+   * @return the manager instance
+   * @throws TException
+   */
+  private PropertyManager getPropertyManager(String ns) throws MetaException, NoSuchObjectException {
+    PropertyStore propertyStore = getMS().getPropertyStore();
+    PropertyManager mgr = PropertyManager.create(ns, propertyStore);
+    return mgr;
+  }
+  @Override
+  public PropertyGetResponse get_properties(PropertyGetRequest req) throws MetaException, NoSuchObjectException, TException {
+    PropertyManager mgr = getPropertyManager(req.getNameSpace());
+    Map<String, PropertyMap> selected = mgr.selectProperties(req.getMapPrefix(), req.getMapPredicate(), req.getMapSelection());

Review Comment:
   Should we convert the `PropertyException`, `JexlException` or others to `MetaException` here so that the client can tell the real cause?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187435115


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -0,0 +1,307 @@
+/* * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonSyntaxException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyManager;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.Source;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.Part;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * The property  cli servlet.
+ */
+public class PropertyServlet extends HttpServlet {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyServlet.class);
+  /** The object store. */
+  private final RawStore objectStore;
+  /** The security. */
+  private final ServletSecurity security;
+
+  PropertyServlet(Configuration configuration, RawStore store) {

Review Comment:
   This is not the same as Thrift over HTTP since this does *not* use Thrift; the messages used by this servlet are JSON encoded.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1542775177

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [121 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib closed pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib closed pull request #4194: HIVE-27186: A persistent property store
URL: https://github.com/apache/hive/pull/4194


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] nrg4878 commented on pull request #4194: HIVE-27186

Posted by "nrg4878 (via GitHub)" <gi...@apache.org>.
nrg4878 commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1495136665

   @vihangk1 @pvary @zabetak @kasakrisz @dengzhhu653 Could I please request a code review for this feature in HMS? There are unit tests as well. Any help much appreciated. Thank you in advance


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1158322429


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/JdoPropertyStore.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.commons.jexl3.JexlException;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertySchema;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * Implementation of the property store delegating persistence to a (jdo) raw store.
+ */
+public class JdoPropertyStore extends PropertyStore {
+  /** The jdo objects store. */
+  private final RawStore objectStore;
+
+  /**
+   * Basic ctor.
+   * @param store the object store
+   */
+  public JdoPropertyStore(RawStore store) {
+    this.objectStore = store;
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.getProperties(mapKey, getPropertyMapFunction(null, getSchema));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public Map<String, PropertyMap> selectProperties(final String keyPrefix, Predicate<String> keyFilter, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.selectProperties(keyPrefix, getPropertyMapFunction(keyFilter, getSchema));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public UUID fetchDigest(String mapKey) {
+    try {
+      return objectStore.getProperties(mapKey, (mm) -> UUID.fromString(mm.getPropertyValue()));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter) {
+    try {
+      return objectStore.selectProperties(keyPrefix, (mm) -> {
+        if (keyFilter == null || keyFilter.test(mm.getPropertykey())) {
+          return UUID.fromString(mm.getPropertyValue());
+        }
+        return null;
+      });
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public void saveProperties(Iterator<Map.Entry<String, PropertyMap>> save) {
+    // will run the super method in a transaction
+    try {
+      objectStore.runInTransaction(()-> super.saveProperties(save));
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  protected void saveProperties(String mapKey, PropertyMap map) {
+    try {
+      if (map.isDropped()) {
+        objectStore.dropProperties(mapKey);
+      } else {
+        objectStore.putProperties(mapKey, map.getDigest().toString(), null, serialize(map));
+      }
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override public boolean dropProperties(String mapKey) {
+   try {
+     return objectStore.dropProperties(mapKey);
+   } catch (MetaException e) {
+     throw new PropertyException(e);
+   }
+  }
+
+  @Override public boolean renameProperties(String mapKey, String newKey) {
+    try {
+      return objectStore.renameProperties(mapKey, newKey);
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+  /**
+   * Creates a function that transforms an MMetastoreDBProperties into a PropertyMap.
+   * @param keyFilter a map key filtering predicate that will make the function return null if test fails
+   * @param getSchema the function that solves a schema from a key
+   * @return a function
+   */
+  Function<MMetastoreDBProperties, PropertyMap> getPropertyMapFunction(final Predicate<String> keyFilter, final Function<String, PropertySchema> getSchema) {
+    return (mm) -> {
+      final String key = mm.getPropertykey();

Review Comment:
   Not necessary, just stating the variable won't change after init. In a method returning a lambda, I tried to be extra-clear.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174268417


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java:
##########
@@ -2185,4 +2187,32 @@ Map<String, Map<String, String>> updatePartitionColumnStatisticsInBatch(
   Package findPackage(GetPackageRequest request);
   List<String> listPackages(ListPackageRequest request);
   void dropPackage(DropPackageRequest request);
+
+  /** Persistent Property Management. */
+  default MMetastoreDBProperties putProperties(String key, String value, String description, byte[] content) throws MetaException {
+    return null;
+  }
+
+  default <T> T getProperties(String key, java.util.function.Function<MMetastoreDBProperties, T> transform) throws MetaException {
+    return null;
+  }
+  default <T> Map<String, T> selectProperties(String key, java.util.function.Function<MMetastoreDBProperties, T> transform) throws MetaException {
+    return null;
+  }
+
+  default boolean renameProperties(String mapKey, String newKey) throws MetaException {
+    return false;
+  }
+
+  default boolean dropProperties(String key) throws MetaException {
+    return false;
+  }
+
+  default PropertyStore getPropertyStore() {

Review Comment:
   Yes I can - and will. :-) 



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171263485


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java:
##########
@@ -2185,4 +2187,32 @@ Map<String, Map<String, String>> updatePartitionColumnStatisticsInBatch(
   Package findPackage(GetPackageRequest request);
   List<String> listPackages(ListPackageRequest request);
   void dropPackage(DropPackageRequest request);
+
+  /** Persistent Property Management. */
+  default MMetastoreDBProperties putProperties(String key, String value, String description, byte[] content) throws MetaException {
+    return null;
+  }
+
+  default <T> T getProperties(String key, java.util.function.Function<MMetastoreDBProperties, T> transform) throws MetaException {
+    return null;
+  }
+  default <T> Map<String, T> selectProperties(String key, java.util.function.Function<MMetastoreDBProperties, T> transform) throws MetaException {
+    return null;
+  }
+
+  default boolean renameProperties(String mapKey, String newKey) throws MetaException {
+    return false;
+  }
+
+  default boolean dropProperties(String key) throws MetaException {
+    return false;
+  }
+
+  default PropertyStore getPropertyStore() {

Review Comment:
   The `PropertyStore` is the place of creating/droping/selecting/renaming properties, so can we just add `PropertyStore getPropertyStore()`, and remove other property methods from the RawStore?
   
   



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174228743


##########
standalone-metastore/metastore-server/src/test/resources/org/apache/hadoop/hive/metastore/properties/pol0.json:
##########
@@ -0,0 +1,24 @@
+{

Review Comment:
   What is this file used for? 



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174268417


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java:
##########
@@ -2185,4 +2187,32 @@ Map<String, Map<String, String>> updatePartitionColumnStatisticsInBatch(
   Package findPackage(GetPackageRequest request);
   List<String> listPackages(ListPackageRequest request);
   void dropPackage(DropPackageRequest request);
+
+  /** Persistent Property Management. */
+  default MMetastoreDBProperties putProperties(String key, String value, String description, byte[] content) throws MetaException {
+    return null;
+  }
+
+  default <T> T getProperties(String key, java.util.function.Function<MMetastoreDBProperties, T> transform) throws MetaException {
+    return null;
+  }
+  default <T> Map<String, T> selectProperties(String key, java.util.function.Function<MMetastoreDBProperties, T> transform) throws MetaException {
+    return null;
+  }
+
+  default boolean renameProperties(String mapKey, String newKey) throws MetaException {
+    return false;
+  }
+
+  default boolean dropProperties(String key) throws MetaException {
+    return false;
+  }
+
+  default PropertyStore getPropertyStore() {

Review Comment:
   I'm not sure, the PropertyStore delegates to the RawStore whose actual methods are implemented in the ObjectStore. 



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171273793


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java:
##########
@@ -5665,6 +5678,212 @@ private String getGuidFromDB() throws MetaException {
     return null;
   }
 
+  @Override
+  public boolean runInTransaction(Runnable exec) throws MetaException {
+    boolean success = false;
+    Transaction tx = null;
+    try {
+      if (openTransaction()) {
+        exec.run();
+        success = commitTransaction();
+      }
+    } catch (Exception e) {
+      LOG.warn("Metastore operation failed", e);
+    } finally {
+      rollbackAndCleanup(success, null);
+    }
+    return success;
+  }
+
+  @Override
+  public boolean dropProperties(String key) throws MetaException {
+    boolean success = false;
+    Transaction tx = null;
+    Query query = null;
+    try {
+      if (openTransaction()) {
+        query = pm.newQuery(MMetastoreDBProperties.class, "this.propertyKey == key");
+        query.declareParameters("java.lang.String key");
+        Collection<MMetastoreDBProperties> properties = (Collection<MMetastoreDBProperties>) query.execute(key);
+        if (!properties.isEmpty()) {
+          pm.deletePersistentAll(properties);
+        }
+        success = commitTransaction();
+      }
+    } catch (Exception e) {
+      LOG.warn("Metastore property drop failed", e);
+    } finally {
+      rollbackAndCleanup(success, query);
+    }
+    return success;
+  }
+
+  @Override
+  public MMetastoreDBProperties putProperties(String key, String value, String description,  byte[] content) throws MetaException {
+    boolean success = false;
+    try {
+      if (openTransaction()) {
+        //pm.currentTransaction().setOptimistic(false);
+        // fetch first to determine new vs update
+        MMetastoreDBProperties properties = doGetProperties(key, null);
+        final boolean newInstance;
+        if (properties == null) {
+          newInstance = true;
+          properties = new MMetastoreDBProperties();
+          properties.setPropertykey(key);
+        } else {
+          newInstance = false;
+        }
+        properties.setDescription(description);
+        properties.setPropertyValue(value);
+        properties.setPropertyContent(content);
+        LOG.debug("Attempting to add property {} for the metastore db", key);
+        properties.setDescription("Metastore property "
+            + (newInstance ? "created" : "updated")
+            + " " + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")));
+        if (newInstance) {
+          pm.makePersistent(properties);
+        }
+        success = commitTransaction();
+        if (success) {
+          LOG.info("Metastore property {} created successfully", key);
+          return properties;
+        }
+      }
+    } catch (Exception e) {
+      LOG.warn("Metastore property save failed", e);
+    } finally {
+      rollbackAndCleanup(success, null);
+    }
+    return null;
+  }
+
+  @Override
+  public boolean renameProperties(String mapKey, String newKey) throws MetaException {
+    boolean success = false;
+    Transaction tx = null;
+    Query query = null;

Review Comment:
   there is a `QueryWrapper` that can wrap the Query, it makes try-with-resources easy to use for `Query`



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171276457


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Serializer.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.properties;
+
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.file.Files;
+
+/**
+ * A helper class to read/write objects through the SerializationProxy.
+ *
+ * @param <T>
+ */
+public class Serializer<T extends Serializable> {
+  private static final Logger LOGGER = SerializationProxy.LOGGER;
+

Review Comment:
   There is a `SerializationUtilities` for the same purpose, can we reuse this utils?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171713557


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/CachingPropertyStore.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.properties;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * A property map store using a pull-thru cache.
+ * Before a map is returned, a check against the stored corresponding digest is performed to avoid
+ * using stale data.
+ */
+public class CachingPropertyStore extends PropertyStore {
+  private static final int MAP_CACHE = 64;
+  protected  SoftCache<String, PropertyMap> maps;
+  protected PropertyStore store;
+  public CachingPropertyStore(PropertyStore wrap) {
+    this(wrap, MAP_CACHE);
+  }
+
+  public CachingPropertyStore(PropertyStore wrap, int capacity) {
+    store = wrap;
+    maps = new SoftCache<>(capacity, false);
+  }
+  public void clearCache() {
+    maps.clear();
+  }
+
+  @Override public UUID fetchDigest(String mapKey) {
+    return store.fetchDigest(mapKey);
+  }
+
+  @Override
+  public Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter) {
+    return store.selectDigest(keyPrefix, keyFilter);
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, final Function<String, PropertySchema> getSchema) {
+    synchronized(this) {
+      PropertyMap map = maps.compute(mapKey, mapsCompute(mapKey, getSchema));
+      // we always return a copy of the properties in the cache
+      return map != null? map.copy() : null;
+    }
+  }
+
+  BiFunction<String, PropertyMap, PropertyMap> mapsCompute(String string, Function<String, PropertySchema> getSchema) {

Review Comment:
   Should the `string ` argument be changed into something more meaningful?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1527911826

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [103 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187282071


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyMap.java:
##########
@@ -0,0 +1,466 @@
+/*
+ * 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.properties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * A property map pertaining to a given object type (cluster, database, table).
+ * <p>
+ *   Maps follow a copy-on-write scheme gated by a dirty flag (avoid copy of a dirty map). This allows
+ *   sharing their content (the inner properties map) with guaranteed isolation and thread safety.
+ * </p>
+ */
+public class PropertyMap implements Serializable {

Review Comment:
   nit: may be we can declare a super class for de/serializing instead extends `Serializable`, for example, the super class might look like:
   ```java
   abstract class Abc {
     public Abc(DataInput input, Object... args) {
       // noop
     }
     public abstract Abc write(DataOutput out);
   }



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187466903


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -0,0 +1,307 @@
+/* * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonSyntaxException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyManager;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.Source;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.Part;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * The property  cli servlet.
+ */
+public class PropertyServlet extends HttpServlet {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyServlet.class);
+  /** The object store. */
+  private final RawStore objectStore;
+  /** The security. */
+  private final ServletSecurity security;
+
+  PropertyServlet(Configuration configuration, RawStore store) {

Review Comment:
   Why not use the HTTP over Thrift directly, we can start the HMS easily by altering a property, for example:
   `TestRemoteHiveHttpMetaStore`
   
   



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1209882648


##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/properties/HMSServletTest.java:
##########
@@ -63,21 +67,21 @@
 
 public class HMSServletTest extends HMSTestBase {

Review Comment:
   There are some test classes for the same purpose, i.e, the HiveServletTest1A, HiveServletTestA, HiveServletTest1, HiveServletTest, maybe we can just keep only one of them.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1567548122

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [93 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1557837588

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [103 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1560910063

   Looks like there are some class conflict in the last Jenkins run:
   ```
   org.apache.commons.codec.binary.Base64.<init>(I[BZ)V
   Stacktrace
   java.lang.NoSuchMethodError: org.apache.commons.codec.binary.Base64.<init>(I[BZ)V
   	at org.apache.hadoop.hive.metastore.security.TestTokenStoreDelegationTokenSecretManager.testRenewal(TestTokenStoreDelegationTokenSecretManager.java:81)


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1575013367

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [90 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1575650966

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [90 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171687275


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -2769,6 +2802,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   GetPartitionsByNamesResult get_partitions_by_names_req(1:GetPartitionsByNamesRequest req)
                         throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
+    // retrieve properties
+    PropertyGetResponse get_properties(1:PropertyGetRequest req);

Review Comment:
   I too agree with @dengzhhu653 



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171268251


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/JdoPropertyStore.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.commons.jexl3.JexlException;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertySchema;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * Implementation of the property store delegating persistence to a (jdo) raw store.
+ */
+public class JdoPropertyStore extends PropertyStore {
+  /** The jdo objects store. */
+  private final RawStore objectStore;
+
+  /**
+   * Basic ctor.
+   * @param store the object store
+   */
+  public JdoPropertyStore(RawStore store) {
+    this.objectStore = store;
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.getProperties(mapKey, getPropertyMapFunction(null, getSchema));

Review Comment:
   we can move the logic of `objectStore.getProperties` to here. For example:
   ```
   JdoPropertyStore(RawStore store, PersistenceManager pm) 
   
   store.openTransaction()
   try {
     // logic
    store.commitTransaction
   }  finally {
     //... 
   }
   ```



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171268251


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/JdoPropertyStore.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.commons.jexl3.JexlException;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertySchema;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * Implementation of the property store delegating persistence to a (jdo) raw store.
+ */
+public class JdoPropertyStore extends PropertyStore {
+  /** The jdo objects store. */
+  private final RawStore objectStore;
+
+  /**
+   * Basic ctor.
+   * @param store the object store
+   */
+  public JdoPropertyStore(RawStore store) {
+    this.objectStore = store;
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.getProperties(mapKey, getPropertyMapFunction(null, getSchema));

Review Comment:
   we can move the logic of `objectStore.getProperties` to here



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171695068


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java:
##########
@@ -5192,4 +5192,25 @@ public List<WriteEventInfo> getAllWriteEventInfo(GetAllWriteEventInfoRequest req
   public AbortCompactResponse abortCompactions(AbortCompactionRequest request) throws TException{
     return client.abort_Compactions(request);
   }
+
+  @Override
+  public boolean setProperties(String nameSpace, Map<String, String> properties) throws TException {
+    PropertySetRequest psr = new PropertySetRequest();
+    psr.setNameSpace(nameSpace);
+    psr.setPropertyMap(properties);
+    return client.set_properties(psr);
+  }
+
+  @Override
+  public Map<String, Map<String, String>> getProperties(String nameSpace, String mapPrefix, String mapPredicate, String... selection) throws TException {
+    PropertyGetRequest request = new PropertyGetRequest();
+    request.setNameSpace(nameSpace);
+    request.setMapPrefix(mapPrefix);
+    request.setMapPredicate(mapPredicate);
+    if (selection != null && selection.length > 0) {
+      request.setMapSelection(Arrays.asList(selection));
+    }
+    PropertyGetResponse response = client.get_properties(request);

Review Comment:
   should we do this instead `client.get_properties(request).getProperties()`?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171286901


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();

Review Comment:
   This `dirtyMaps` sounds a bit overlap with `CachingPropertyStore`



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171274696


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MMetastoreDBProperties.java:
##########
@@ -21,6 +21,8 @@ public class MMetastoreDBProperties {
   private String propertyKey;
   private String propertyValue;
   private String description;
+  private byte[] propertyContent;

Review Comment:
   Should we create a column in underlying database that maps to this field?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1505757045

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [85 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174259080


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Serializer.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.properties;
+
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.file.Files;
+
+/**
+ * A helper class to read/write objects through the SerializationProxy.
+ *
+ * @param <T>
+ */
+public class Serializer<T extends Serializable> {
+  private static final Logger LOGGER = SerializationProxy.LOGGER;
+

Review Comment:
   I'll look into it; the Serializer/SerializationProxy allows a very clean control and very compact representation.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175580801


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -75,22 +78,7 @@
 import java.lang.reflect.UndeclaredThrowableException;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
-import java.util.AbstractMap;

Review Comment:
   If the import is not used, it is better to remove it (Sonar/CheckStyle do too). 



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174275569


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java:
##########
@@ -4393,4 +4393,27 @@ ReplicationMetricList getReplicationMetrics(GetReplicationMetricsRequest
 
   AbortCompactResponse abortCompactions(AbortCompactionRequest request) throws TException;
 
+  /**
+   * Sets properties.
+   * @param nameSpace the property store namespace
+   * @param properties a map keyed by property path mapped to property values
+   * @return true if successful, false otherwise
+   * @throws TException
+   */
+  default boolean setProperties(String nameSpace, Map<String, String> properties) throws TException {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Gets properties.
+   * @param nameSpace the property store namespace.
+   * @param mapPrefix the map prefix (ala starts-with) to select maps
+   * @param mapPredicate predicate expression on properties to further reduce the selected maps
+   * @param selection the list of properties to return, null for all
+   * @return a map keyed by property map path to maps keyed by property name mapped to property values
+   * @throws TException
+   */
+  default Map<String, Map<String, String>> getProperties(String nameSpace, String mapPrefix, String mapPredicate, String... selection) throws TException {
+    throw new UnsupportedOperationException();

Review Comment:
   Same, implemented in HiveMetaStoreClient.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174273969


##########
standalone-metastore/metastore-common/pom.xml:
##########
@@ -25,6 +25,11 @@
     <standalone.metastore.path.to.root>..</standalone.metastore.path.to.root>
   </properties>
   <dependencies>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-jexl3</artifactId>
+      <version>3.3</version>

Review Comment:
   You mean declare/use a ${jexl.version} ?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187394625


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -0,0 +1,307 @@
+/* * 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;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonSyntaxException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyManager;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.Source;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.Part;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Reader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * The property  cli servlet.
+ */
+public class PropertyServlet extends HttpServlet {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyServlet.class);
+  /** The object store. */
+  private final RawStore objectStore;
+  /** The security. */
+  private final ServletSecurity security;
+
+  PropertyServlet(Configuration configuration, RawStore store) {

Review Comment:
   As you can see, we have already supported HTTP over Thrift in HMS, if this is an issue, can we track it in a separate Jira? Thanks!



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1188094645


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,641 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.JexlScript;
+import org.apache.commons.jexl3.ObjectContext;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;

Review Comment:
   nit: unused import



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] TuroczyX commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "TuroczyX (via GitHub)" <gi...@apache.org>.
TuroczyX commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1158344677


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/JdoPropertyStore.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.commons.jexl3.JexlException;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertySchema;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * Implementation of the property store delegating persistence to a (jdo) raw store.
+ */
+public class JdoPropertyStore extends PropertyStore {
+  /** The jdo objects store. */
+  private final RawStore objectStore;
+
+  /**
+   * Basic ctor.
+   * @param store the object store
+   */
+  public JdoPropertyStore(RawStore store) {
+    this.objectStore = store;
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.getProperties(mapKey, getPropertyMapFunction(null, getSchema));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public Map<String, PropertyMap> selectProperties(final String keyPrefix, Predicate<String> keyFilter, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.selectProperties(keyPrefix, getPropertyMapFunction(keyFilter, getSchema));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public UUID fetchDigest(String mapKey) {
+    try {
+      return objectStore.getProperties(mapKey, (mm) -> UUID.fromString(mm.getPropertyValue()));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter) {
+    try {
+      return objectStore.selectProperties(keyPrefix, (mm) -> {
+        if (keyFilter == null || keyFilter.test(mm.getPropertykey())) {
+          return UUID.fromString(mm.getPropertyValue());
+        }
+        return null;
+      });
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public void saveProperties(Iterator<Map.Entry<String, PropertyMap>> save) {
+    // will run the super method in a transaction
+    try {
+      objectStore.runInTransaction(()-> super.saveProperties(save));
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  protected void saveProperties(String mapKey, PropertyMap map) {
+    try {
+      if (map.isDropped()) {
+        objectStore.dropProperties(mapKey);
+      } else {
+        objectStore.putProperties(mapKey, map.getDigest().toString(), null, serialize(map));
+      }
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override public boolean dropProperties(String mapKey) {
+   try {
+     return objectStore.dropProperties(mapKey);
+   } catch (MetaException e) {
+     throw new PropertyException(e);
+   }
+  }
+
+  @Override public boolean renameProperties(String mapKey, String newKey) {
+    try {
+      return objectStore.renameProperties(mapKey, newKey);
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+  /**
+   * Creates a function that transforms an MMetastoreDBProperties into a PropertyMap.
+   * @param keyFilter a map key filtering predicate that will make the function return null if test fails
+   * @param getSchema the function that solves a schema from a key
+   * @return a function
+   */
+  Function<MMetastoreDBProperties, PropertyMap> getPropertyMapFunction(final Predicate<String> keyFilter, final Function<String, PropertySchema> getSchema) {
+    return (mm) -> {
+      final String key = mm.getPropertykey();

Review Comment:
   ack



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175580801


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -75,22 +78,7 @@
 import java.lang.reflect.UndeclaredThrowableException;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
-import java.util.AbstractMap;

Review Comment:
   If the import is not used, it is better to remove it (Sonar/CheckStyle think so too). 



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174277376


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Digester.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.properties;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Date;
+import java.util.UUID;
+
+/**
+ * Helper class that creates a type 5 uuid.
+ * <p>This is computed from a set of updates using a SHA-1 message digest massaged into an UUID.
+ * see <a href="https://en.wikipedia.org/wiki/Universally_unique_identifier">...</a>
+ */
+public class Digester {
+    /** The Namespace uuid. */
+    private final UUID nsuid;
+    /** The digest used to compute the UUID. */
+    private final MessageDigest md;
+    /** A default namespace based on the class loading time. */
+    private static final UUID TEMP_NS;
+    static {
+        MessageDigest md = createDigest();
+        digest(md, System.currentTimeMillis());
+        TEMP_NS = computeUUID(md);
+    }
+
+    /**
+     * Allows to update the message digest from an object.
+     */
+    private static class TagOutputStream extends OutputStream {
+        /** The digest to update. */
+        private final MessageDigest md;
+
+        /**
+         * Sole ctor.
+         * @param md the message digester
+         */
+        TagOutputStream(MessageDigest md) {
+            this.md = md;
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            md.update((byte) b);
+        }
+
+        @Override
+        public void write(byte[] b, int off, int len) throws IOException {
+            md.update(b, off, len);
+        }
+    }
+
+    /**
+     * @return a SHA-1 message digest
+     */
+    private static MessageDigest createDigest() {
+        MessageDigest md;
+        try {
+            md = MessageDigest.getInstance("SHA-1");
+        } catch (NoSuchAlgorithmException nsae) {
+            throw new InternalError("SHA not supported");
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with a uuid.
+     * @param md the digest to update
+     * @param uid the uuid
+     */
+    private static MessageDigest digest(MessageDigest md, UUID uid) {
+        if (uid != null) {
+            long msb = uid.getMostSignificantBits();
+            digest(md, msb);
+            long lsb = uid.getLeastSignificantBits();
+            digest(md, lsb);
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with an input stream.
+     * @param md the digest to update
+     * @param input the input to consume
+     * @throws IllegalStateException if an io exception occurs
+     */
+    private static void digest(MessageDigest md, InputStream input) {
+        try (OutputStream out = new TagOutputStream(md)) {
+            byte[] buffer = new byte[1024];
+            int read;
+            while ((read = input.read(buffer, 0, 1024)) >= 0) {
+                out.write(buffer, 0, read);
+            }
+        } catch(IOException xio) {
+            throw new IllegalStateException(xio);
+        }
+    }
+
+
+    /**
+     * Updates a digest with a long.
+     * @param md the digest to update
+     * @param l8 the long
+     */
+    private static void digest(MessageDigest md, long l8) {
+        md.update((byte) (l8 & 0xff));
+        md.update((byte) (l8 >> 8));
+        md.update((byte) (l8 >> 16));
+        md.update((byte) (l8 >> 24));
+        md.update((byte) (l8 >> 32));
+        md.update((byte) (l8 >> 40));
+        md.update((byte) (l8 >> 48));
+        md.update((byte) (l8 >> 56));
+    }
+
+    /**
+     * Updates a digest with an object.
+     * @param md the digest to update
+     * @param obj the object
+     */
+    private static void digest(MessageDigest md, Object obj) {
+        if (obj == null) {
+            return;
+        }
+        try (ObjectOutput out = new ObjectOutputStream(new TagOutputStream(md))) {
+            out.writeObject(obj);
+        } catch (IOException ex) {
+            // ignore close exception
+        }
+        // ignore close exception
+    }
+
+    /**
+     * Computes the uuid.
+     * @param md the message digest used to compute the hash
+     * @return the eTag as a type 5 uuid
+     */
+    private static UUID computeUUID(MessageDigest md) {
+        byte[] sha1Bytes = md.digest();
+        sha1Bytes[6] &= 0x0f;  /* clear version        */
+        sha1Bytes[6] |= 0x50;  /* set to version 5     */
+        sha1Bytes[8] &= 0x3f;  /* clear variant        */
+        sha1Bytes[8] |= 0x80;  /* set to IETF variant  */
+
+        // SHA generates 160 bytes; truncate to 128
+        long msb = 0;
+        //assert data.length == 16 || data.length == 20;
+        for (int i = 0; i < 8; i++) {
+            msb = (msb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        long lsb = 0;
+        for (int i = 8; i < 16; i++) {
+            lsb = (lsb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        return new UUID(msb, lsb);
+    }
+
+    /**
+     * A marker interface for objects that can be digested.
+     */
+    public interface Digestible {
+        /**
+         * Updates a digest with this variable.
+         * @param digester the digester to update
+         * @return true if this digestible actually contributed to the digest
+         */
+        boolean digest(Digester digester);
+    }
+
+    /**
+     * A type 5 uuid is namespace + sha1; namespace in our case is a uuid.
+     * Two instances of digesters built with the same namespace will produce the same UUIDs from the
+     * same inputs.
+     * @param namespace the uuid namespace
+     */
+    public Digester(UUID namespace) {
+        nsuid = namespace == null? TEMP_NS : namespace;
+        md = createDigest();
+        // inject namespace
+        digest(md, nsuid);
+    }
+
+    /**
+     * A copy ctor base.
+     * @param lnsuid the namespace uid
+     * @param lmd the message digest
+     */
+    private Digester(UUID lnsuid, MessageDigest lmd)  {
+        this.nsuid = lnsuid;
+        this.md = lmd;
+    }
+
+    /**
+     * Default ctor.
+     * The created digester uses the class loading time as seed for its namespace; this means 2 instances of digester
+     * built in different JVM instances will *NOT* produce the same UUIDs for the same input. Typical use is in
+     * a non-persistent scenario, to verify an instance of an object has not been modified by checking
+     * its digested UUID remained the same.
+     * To get stable UUID computation across time and space in Digester usable in persistent scenario,
+     * you *NEED* to use a namespace-based digester using {@link Digester(UUID)}, uuid that is easily created
+     * using {@link UUID#nameUUIDFromBytes(byte[])} from any name/uri you might desire.
+     */
+    public Digester() {
+        this(null);
+    }
+
+    /**
+     * @return a clone of this instance
+     */
+    public Digester copy() {
+        try {
+            return new Digester(nsuid, (MessageDigest) md.clone());
+        } catch (CloneNotSupportedException ex) {
+            return null;
+        }
+    }
+
+    /**
+     * Computes the version tag from this digester.
+     * <p>This uses the current message digest state and resets it.
+     * @return the type 5 uuid
+     */
+    public UUID getUUID() {
+        UUID uuid = computeUUID(md);
+        md.reset();
+        digest(nsuid);
+        return uuid;
+    }
+
+    /**
+     * Updates the digest with a boolean.
+     * @param b the boolean
+     * @return this digester
+     */
+    public Digester digest(boolean b) {
+        md.update((byte) (b? 1 : 0));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a char.
+     * @param c the char
+     * @return this digester
+     */
+    public Digester digest(char c) {
+        md.update((byte) (c & 0xff));
+        md.update((byte) (c >> 8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a bytes array.
+     * @param bytes the bytes
+     * @return this digester
+     */
+    public Digester digest(byte[] bytes) {
+        if (bytes != null) {
+            md.update(bytes);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with an integer.
+     * @param i4 the int
+     * @return this digester
+     */
+    public Digester digest(int i4) {
+        md.update((byte) (i4 & 0xff));
+        md.update((byte) (i4 >> 8));
+        md.update((byte) (i4 >> 16));
+        md.update((byte) (i4 >> 24));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a long.
+     * @param l8 the long
+     * @return this digester
+     */
+    public Digester digest(long l8) {
+        digest(md, l8);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a double.
+     * @param f8 the double
+     * @return this digester
+     */
+    public Digester digest(double f8) {
+        digest(md, Double.doubleToRawLongBits(f8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a date.
+     * @param date the date
+     * @return this digester
+     */
+    public Digester digest(Date date) {
+        if (date != null) {
+            digest(md, date.getTime());
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a string.
+     * @param str the string
+     * @return this digester
+     */
+    public Digester digest(String str) {
+        if (str != null) {
+            final int sz = str.length();
+            for(int i = 0; i < sz; ++i) {
+                digest(str.charAt(i));
+            }
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uid the uuid
+     * @return this digester
+     */
+    public Digester digest(UUID uid) {
+        digest(md, uid);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uri the uri
+     * @return this digester
+     */
+    public Digester digest(URI uri) {
+        digest(md, uri.toString());
+        return this;
+    }
+
+    /**
+     * Updates the digest with an object that describes how it digests.
+     * @param digestible the object
+     * @return this digester
+     */
+    public Digester digest(Digestible digestible) {
+        if (digestible != null) {
+            digestible.digest(this);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a stream.
+     * @param input the stream
+     * @return this digester
+     */
+    public Digester digest(InputStream input) {
+        if (input != null) {
+            digest(md, input);
+        }
+        return this;
+    }
+    /**
+     * Updates the digest with any (serializable) object.
+     * @param obj the object
+     * @return this digester
+     */
+    public Digester digest(Object obj) {
+        if (obj instanceof Digestible) {

Review Comment:
   Might try and switch on the class name in some concrete cases but would still need a few 'ifs'. Besides, Digestible classes and/or direct calls make this generic method rarely needed.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174270928


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();

Review Comment:
   No; they maintain the updates isolated whilst the cache can serve other (concurrent) reads.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174278355


##########
standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/properties/PocExternal.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.properties;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+
+/**
+ * A Proof Of Concept about externalizable.
+ */
+public class PocExternal implements Serializable {

Review Comment:
   I like unit tests that prove things in isolation.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174283306


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java:
##########
@@ -4393,4 +4393,27 @@ ReplicationMetricList getReplicationMetrics(GetReplicationMetricsRequest
 
   AbortCompactResponse abortCompactions(AbortCompactionRequest request) throws TException;
 
+  /**
+   * Sets properties.
+   * @param nameSpace the property store namespace
+   * @param properties a map keyed by property path mapped to property values
+   * @return true if successful, false otherwise
+   * @throws TException
+   */
+  default boolean setProperties(String nameSpace, Map<String, String> properties) throws TException {
+    throw new UnsupportedOperationException();

Review Comment:
   SessionHiveMetaStoreClient extends HiveMetaStoreClient where the default methods are overridden and implemented. 



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171294897


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -167,6 +167,39 @@ const string HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__"
 const string HIVE_FILTER_FIELD_TABLE_NAME = "hive_filter_field_tableName__"
 const string HIVE_FILTER_FIELD_TABLE_TYPE = "hive_filter_field_tableType__"
 
+/** Table maintenance operation type. */
+enum MaintenanceOpType {
+    COMPACTION = 1,
+    SNAPSHOT_EXPIRY = 2,
+    STATS_REBUILD= 3,
+    MV_BUILD= 4,

Review Comment:
   What's the `MaintenanceOpType` and `MaintenanceOpStatus` for?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171263485


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java:
##########
@@ -2185,4 +2187,32 @@ Map<String, Map<String, String>> updatePartitionColumnStatisticsInBatch(
   Package findPackage(GetPackageRequest request);
   List<String> listPackages(ListPackageRequest request);
   void dropPackage(DropPackageRequest request);
+
+  /** Persistent Property Management. */
+  default MMetastoreDBProperties putProperties(String key, String value, String description, byte[] content) throws MetaException {
+    return null;
+  }
+
+  default <T> T getProperties(String key, java.util.function.Function<MMetastoreDBProperties, T> transform) throws MetaException {
+    return null;
+  }
+  default <T> Map<String, T> selectProperties(String key, java.util.function.Function<MMetastoreDBProperties, T> transform) throws MetaException {
+    return null;
+  }
+
+  default boolean renameProperties(String mapKey, String newKey) throws MetaException {
+    return false;
+  }
+
+  default boolean dropProperties(String key) throws MetaException {
+    return false;
+  }
+
+  default PropertyStore getPropertyStore() {

Review Comment:
   The `PropertyStore` is the place of creating/droping/selecting/renaming properties, so can we just add `PropertyStore getPropertyStore()`, and remove other properties methods from the RawStore?
   
   



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1177647457


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/SoftCache.java:
##########
@@ -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.hadoop.hive.metastore.properties;
+
+import java.lang.ref.SoftReference;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * A soft referenced cache.
+ * <p>
+ * The actual cache is held through a soft reference, allowing it to be GCed under memory pressure.</p>
+ * <p>
+ * This class is <em>not</em> thread-safe.</p>
+ * @param <K> the cache key entry type
+ * @param <V> the cache key value type
+ */
+public class SoftCache<K, V> {
+    /** The default cache capacity. */
+    private static final int CACHE_CAPACITY = 64;
+    /** The default cache load factor. */
+    private static final float LOAD_FACTOR = 0.75f;

Review Comment:
   Exposed through conf (MetastoreConf.ConfVars.PROPERTIES_CACHE_CAPACITY, MetastoreConf.ConfVars.PROPERTIES_CACHE_LOADFACTOR)



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1524015724

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [103 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1534864126

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [112 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187361181


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyMap.java:
##########
@@ -0,0 +1,466 @@
+/*
+ * 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.properties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/**
+ * A property map pertaining to a given object type (cluster, database, table).
+ * <p>
+ *   Maps follow a copy-on-write scheme gated by a dirty flag (avoid copy of a dirty map). This allows
+ *   sharing their content (the inner properties map) with guaranteed isolation and thread safety.
+ * </p>
+ */
+public class PropertyMap implements Serializable {

Review Comment:
   We could but this is not needed and would hide the convention which does not require any extending any class, just implement the interface. And note that most serialization methods are private (
     private Object writeReplace() throws ObjectStreamException; private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1204973416


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PropertyServlet.java:
##########
@@ -194,34 +217,34 @@ private void runPost(HttpServletRequest request,
 
   @Override
   protected void doPut(HttpServletRequest request,
-                        HttpServletResponse response) throws ServletException, IOException {
+                       HttpServletResponse response) throws ServletException, IOException {
     security.execute(request, response, PropertyServlet.this::runGet);
   }
   private void runGet(HttpServletRequest request,
                        HttpServletResponse response) throws ServletException, IOException {
-
-    String ns = getNamespace(request.getRequestURI());
-    PropertyManager mgr;
+    final RawStore ms =  getMS();
+    final String ns = getNamespace(request.getRequestURI());
+    final PropertyManager mgr = getPropertyManager(ms, ns);
     try {
-      mgr = getPropertyManager(ns);
-    } catch (MetaException | NoSuchObjectException exception) {
-      throw new ServletException(exception);
-    }
-    Object json = readJson(request);
-    if (json instanceof Map) {
-      try {
-        @SuppressWarnings("unchecked")
-        Map<String, ?> cast = (Map<String, ?>) json;
-        mgr.setProperties(cast);
-        mgr.commit();
-        response.setStatus(HttpServletResponse.SC_OK);
-        return;
-      } catch (Exception any) {
-        mgr.rollback();
-        response.sendError(HttpServletResponse.SC_BAD_REQUEST, "select fail " + any);
+      Object json = readJson(request);
+      if (json instanceof Map) {
+        try {
+          @SuppressWarnings("unchecked")
+          Map<String, ?> cast = (Map<String, ?>) json;
+          mgr.setProperties(cast);
+          mgr.commit();
+          response.setStatus(HttpServletResponse.SC_OK);
+          return;
+        } catch (Exception any) {
+          mgr.rollback();
+          LOGGER.error("select fail", any);

Review Comment:
   nit: should it better to rename the method to `runPut`, also the error message seems a bit confusing.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1575121821

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [90 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1204602527


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyStore.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.properties;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * The PropertyStore is the persistent container of property maps.
+ * Maps are addressed in the store by their key -  their name prepended by their manager&quot;s namespace.
+ */
+
+public abstract class PropertyStore {
+  /**
+   * Fetches a property map.
+   * @param mapKey the map key
+   * @param getSchema the method to retrieve a schema if the map needs to be created
+   * @return the item property map
+   */
+  public abstract PropertyMap fetchProperties(String mapKey, Function<String, PropertySchema> getSchema);
+
+
+  /**
+   * Fetches a map of property maps.
+   * @param keyPrefix the map key prefix
+   * @param keyFilter a filter for map keys
+   * @param getSchema the method to retrieve a schema if the map needs to be created
+   * @return the map of property map
+   */
+  public abstract Map<String, PropertyMap> selectProperties(final String keyPrefix, Predicate<String> keyFilter, Function<String, PropertySchema> getSchema);
+  /**
+   * Fetches a property map digest.
+   * @param mapKey the map key
+   * @return the item property map
+   */
+  public abstract UUID fetchDigest(String mapKey);
+
+  /**
+   * Fetches a map of property maps digest keyed by their name.
+   * @param keyPrefix the map key prefix
+   * @param keyFilter a filter for map keys
+   * @return the map of property map digests
+   */
+  public abstract Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter);
+
+    /**
+     * Persists a property map.
+     * @param mapKey the map key
+     * @param map the map instance
+     */
+  protected abstract void saveProperties(String mapKey, PropertyMap map);
+
+
+  /**
+   * Drops a property map.
+   * @param mapKey the map key
+   */
+  protected abstract boolean dropProperties(String mapKey);
+
+  /**
+   * Renames a property map.
+   * @param mapKey the map source key
+   * @param newKey the new target key
+   */
+  public abstract boolean renameProperties(String mapKey, String newKey);
+
+  /**
+   * Persists an iterator property map.
+   * <p>May be useful to override to use one transaction.</p>
+   * @param save the iterator on pairs for map key, property map
+   */
+  public void saveProperties(Iterator<Map.Entry<String, PropertyMap>> save) {
+    while(save.hasNext()) {
+      Map.Entry<String, PropertyMap> pair = save.next();
+      PropertyMap map = pair.getValue();
+      if (map != null) {
+        saveProperties(pair.getKey(), map);
+      } else {
+        dropProperties(pair.getKey());
+      }
+    }
+  }
+
+  /**
+   * Serializes a map as a byte array.
+   * @param map the (nonnull) map to write
+   * @return the byte array
+   */
+  public byte[] serialize(PropertyMap map) {
+    return SerializationProxy.toBytes(map);
+  }
+
+  /**
+   * Deserializes a map from a byte array.
+   * @param bytes the byte array
+   * @return the (nonnull) oroperty map
+   */
+  public PropertyMap deserialize(byte[] bytes, Function<String, PropertySchema> getSchema) {
+    return SerializationProxy.fromBytes(bytes, this, getSchema);
+  }
+
+  /**
+   * Default ctor.
+   */
+  protected PropertyStore() {
+  }
+
+  /**
+   * A non-persistent store, for tests mainly.
+   */
+  public static class Transient extends PropertyStore {

Review Comment:
   It could. It most likely will. :-)



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1188094363


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/CachingPropertyStore.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.PROPERTIES_CACHE_CAPACITY;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.PROPERTIES_CACHE_LOADFACTOR;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * A property map store using a pull-thru cache.
+ * <p>
+ * Before a map is returned, a check against the stored corresponding digest is performed to avoid
+ * using stale data.
+ * </p>
+ */
+public class CachingPropertyStore extends PropertyStore {
+  protected final SoftCache<String, PropertyMap> maps;
+  protected final PropertyStore store;
+  public CachingPropertyStore(PropertyStore wrap) {
+    this(wrap, new Configuration());
+  }
+
+  public CachingPropertyStore(PropertyStore wrap, Configuration conf) {
+    store = wrap;
+    int capacity = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.PROPERTIES_CACHE_CAPACITY);
+    double fillFactor = MetastoreConf.getDoubleVar(conf, MetastoreConf.ConfVars.PROPERTIES_CACHE_LOADFACTOR);
+    maps = new SoftCache<>(capacity, fillFactor, false);
+  }
+  public void clearCache() {
+    maps.clear();
+  }
+
+  @Override public UUID fetchDigest(String mapKey) {
+    return store.fetchDigest(mapKey);
+  }
+
+  @Override
+  public Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter) {
+    return store.selectDigest(keyPrefix, keyFilter);
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, final Function<String, PropertySchema> getSchema) {
+    synchronized(this) {
+      PropertyMap map = maps.compute(mapKey, mapsCompute(mapKey, getSchema));
+      // we always return a copy of the properties in the cache
+      return map != null? map.copy() : null;
+    }
+  }
+
+  BiFunction<String, PropertyMap, PropertyMap> mapsCompute(String string, Function<String, PropertySchema> getSchema) {
+    return (k, v) -> {
+      PropertyMap map = v;
+      if (map != null) {
+        UUID digest = map.getDigest();
+        UUID fetchedDigest = fetchDigest(string);
+        if (fetchedDigest != null && !Objects.equals(digest, fetchedDigest)) {
+          map = null;
+        }
+      }
+      if (map == null) {
+        map = store.fetchProperties(string, getSchema);
+      }
+      return map;
+    };
+  }
+
+  @Override
+  public Map<String, PropertyMap> selectProperties(final String keyPrefix, Predicate<String> keyFilter, Function<String, PropertySchema> getSchema) {
+    final Map<String, PropertyMap> results = new TreeMap<>();
+    // go select the digests for the maps we seek
+    final Map<String, UUID> digests = store.selectDigest(keyPrefix, keyFilter);
+    final Iterator<Map.Entry<String, UUID>> idigest = digests.entrySet().iterator();
+    while (idigest.hasNext()) {
+      Map.Entry<String, UUID> entry = idigest.next();
+      String key = entry.getKey();
+      PropertyMap map = maps.get(key);
+      // remove from maps to select and add to results if in the cache and digest is valid
+      if (map != null && Objects.equals(map.getDigest(), entry.getValue())) {
+        results.put(key, map.copy());
+        idigest.remove();
+      }
+    }
+    // digests now contains the names of maps required that are not results
+    Map<String, PropertyMap> selectedMaps = store.selectProperties(keyPrefix, digests::containsKey, getSchema);
+    // we cache those new maps and for each add the copy to the result if we have not loaded and cached it concurrently
+    selectedMaps.forEach((k, v) -> {
+      PropertyMap m = maps.putIfAbsent(k, v);
+      results.put(k, m != null && m.isDirty()? m : v.copy());
+    });
+    return results;
+  }
+
+  @Override
+  public void saveProperties(String mapKey, PropertyMap map) {
+    synchronized(this) {
+      store.saveProperties(mapKey, map);
+      maps.put(mapKey, map);
+    }
+  }
+
+  @Override
+  protected boolean dropProperties(String mapKey) {
+    synchronized(this) {
+      boolean b = store.dropProperties(mapKey);
+      maps.clear();
+      return b;
+    }
+  }
+
+  @Override
+  public boolean renameProperties(String mapKey, String newKey) {
+    synchronized (this) {
+      // target is unencumbered
+      if (!maps.containsKey(newKey)) {
+        PropertyMap map = maps.remove(mapKey);
+        // we got a source
+        if (map != null) {
+          maps.put(newKey, map);
+          return true;
+        }

Review Comment:
   nit: Should we also rename the properties stored in db?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1187438016


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,629 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.JexlScript;
+import org.apache.commons.jexl3.ObjectContext;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ * <p>The manager class needs to be registered with its namespace as key</p>
+ * <p>
+ *   Since a collection of properties are stored in a map, to avoid hitting the persistence store for each update
+ *   - which would mean rewriting the map multiple times - the manager keeps track of dirty maps whilst
+ *   serving as transaction manager. This way, when importing multiple properties targeting different elements (think
+ *   setting properties for different tables), each impacted map is only rewritten
+ *   once by the persistence layer during commit. This also allows multiple calls to participate to one transactions.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected final Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property manager class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   * @throws MetaException if the manager creation fails
+   * @throws NoSuchObjectException if the store is null or no constructor was declared
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) throws MetaException, NoSuchObjectException {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized (NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor == null) {
+      throw new NoSuchObjectException("no PropertyManager namespace is declared, namespace " + namespace);
+    }
+    if (store == null) {
+      throw new NoSuchObjectException("no PropertyStore exists " + namespace);
+    }
+    try {
+      return ctor.newInstance(namespace, store);
+    } catch (Exception xany) {
+      LOGGER.error("PropertyManager creation failed " + namespace, xany);
+      throw new MetaException("PropertyManager creation failed, namespace " + namespace);
+    }
+  }
+
+  /**
+   * JEXL adapter.
+   * <p>public for introspection.</p>
+   */
+  public static class MapWrapper implements JexlContext {
+    PropertyMap map;
+    MapWrapper(PropertyMap map) {
+      this.map = map;
+    }
+
+    public Object get(String p) {
+      return map.getPropertyValue(p);
+    }
+
+    @Override
+    public void set(String name, Object value) {
+      map.putProperty(name, value);
+    }
+
+    @Override
+    public boolean has(String name) {
+      return map.getTypeOf(name) != null;
+    }
+  }
+
+  /**
+   * Creates a manager instance.
+   * @param store the store instance which must use an appropriate property map factory (probably use createMap).
+   */
+  protected PropertyManager(String ns, PropertyStore store) {
+    this.namespace = ns;
+    this.store = store;
+  }
+
+  /**
+   * Saves all pending updates to store.
+   */
+  public void commit() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      if (!dirtyMaps.isEmpty()) {
+        store.saveProperties(dirtyMaps.entrySet().iterator());
+        dirtyMaps.clear();
+      }
+    }
+  }
+
+  /**
+   * Forget all pending updates.
+   */
+  public void rollback() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      dirtyMaps.clear();
+    }
+  }
+
+  /**
+   * Imports a set of default values into this store&quot;s schema.
+   * The properties should be of the form schema_name.property_name=value.
+   * Note that this implies the manager has at least one known property map schema.
+   * @param importsp the properties
+   */
+  public void importDefaultValues(Properties importsp) {
+    importsp.forEach((k, v)->{
+      String importName = k.toString();
+      final int dotPosition = importName.indexOf(".");
+      if (dotPosition > 0) {
+        String schemaName = importName.substring(0, dotPosition);
+        PropertySchema schema = getSchema(schemaName);
+        if (schema != null) {
+          String propertyName = importName.substring(dotPosition + 1);
+          schema.setDefaultValue(propertyName, v);
+        }
+      }
+    });
+  }
+
+  /**
+   * Imports a set of property values.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param map the properties key=value
+   */
+  public void setProperties(Properties map) {
+    map.forEach((k, v)-> setProperty(k.toString(), v));
+  }
+
+  /**
+   * Injects a set of properties.
+   * If the value is null, the property is removed.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param map the map of properties to inject.
+   */
+  public void setProperties(Map<String, ?> map) {
+    map.forEach(this::setProperty);
+  }
+
+  /**
+   * Sets a property value.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param key the property key
+   * @param value the property value or null to unset
+   */
+  public void setProperty(String key, Object value) {
+    setProperty(splitKey(key), value);
+  }
+
+  /**
+   * Runs a JEXL script using this manager as context.
+   * @param src the script source
+   * @return the script result
+   * @throws PropertyException if any error occurs in JEXL
+   */
+  public Object runScript(String src) throws PropertyException {
+    try {
+      JexlScript script = JEXL.createScript(src);
+      ObjectContext<PropertyManager> context = new ObjectContext<>(JEXL, this);
+      return script.execute(context);
+    } catch(JexlException je) {
+      throw new PropertyException("script failed", je);
+    }
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or null if not assigned
+   */
+  public Object getProperty(String key) {
+    return getProperty(splitKey(key));
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or the schema default value if not assigned
+   */
+  public Object getPropertyValue(String key) {
+    return getPropertyValue(splitKey(key));
+  }
+
+  /**
+   * Splits a property key into its fragments.
+   * @param key the property key
+   * @return the key fragments
+   */
+  protected String[] splitKey(String key) {
+    String[] splits = key.split("(?<!\\\\)\\.");
+    if (splits.length < 1) {
+      splits = new String[]{key};
+    }
+    return splits;
+  }
+
+  /**
+   * Gets a schema by name.
+   * <p>Only used by {@link #importDefaultValues(Properties)}</p>
+   * @param name schema name
+   * @return the schema instance, null if no such schema is known
+   */
+  public PropertySchema getSchema(String name) {
+    return null;
+  }
+
+  /**
+   * Determines the schema from the property key fragments.
+   * @param keys the key fragments
+   * @return the schema, {@link PropertySchema#NONE} if no such schema is known
+   */
+  protected PropertySchema schemaOf(String[] keys) {
+    return PropertySchema.NONE;
+  }
+
+  /**
+   * @param keys property key fragments
+   * @return number of fragments composing the map name in the fragments array
+   */
+  protected int getMapNameLength(String[] keys) {
+    return keys.length - 1;
+  }
+
+  /**
+   * Compose a property map key from a property map name.
+   * @param name the property map name, may be null or empty
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String name) {
+    StringBuilder strb = new StringBuilder(namespace);
+    if (name != null && !name.isEmpty()){
+      strb.append('.');
+      strb.append(name);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Extract a property map name from a property map key.
+   * @param key property map key
+   * @return the property map name
+   */
+  protected String mapName(String key) {
+    int dot = key.indexOf('.');
+    return dot > 0? key.substring(dot + 1) : key;
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the key fragments
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String[] keys) {
+    return mapKey(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the property key fragments
+   * @param maxkl the maximum number of fragments in the map key
+   * @return the property key used by the store
+   */
+  protected String mapKey(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    // shortest map key is namespace
+    StringBuilder strb = new StringBuilder(namespace);
+    for(int k = 0; k < Math.min(maxkl, keys.length - 1); ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @return the property name
+   */
+  protected String propertyName(String[] keys) {
+    return propertyName(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @param maxkl the maximum number of fragments in the map name
+   * @return the property name
+   */
+  protected String propertyName(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    if (keys.length <= maxkl) {
+      return keys[keys.length - 1];
+    }
+    StringBuilder strb = new StringBuilder(keys[maxkl]);
+    for(int k = maxkl + 1; k < keys.length; ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or null if none was assigned
+   */
+  public Object getProperty(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    if (map == null) {
+      map = store.fetchProperties(mapKey, null);
+    }
+    if (map != null) {
+      return map.getProperty(propertyName(keys));
+    }
+    return null;
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or the default schema value if not assigned
+   */
+  public Object getPropertyValue(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    PropertySchema schema = schemaOf(keys);
+    if (map == null) {
+      map = store.fetchProperties(mapKey, s->schema);
+    }
+    String propertyName = propertyName(keys);
+    if (map != null) {
+      return map.getPropertyValue(propertyName);
+    }
+    if (schema != null) {
+      return schema.getDefaultValue(propertyName);
+    }
+    return null;
+  }
+
+  /**
+   * Drops a property map.
+   * <p>Transactional call that requires calling {@link #commit()} or {@link #rollback()}.</p>
+   * @param mapName the map name
+   * @return true if the properties may exist, false if they did nots
+   */
+  public boolean dropProperties(String mapName) {
+    final String mapKey = mapKey(mapName);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null && Objects.equals(PropertyMap.DROPPED, dirtyMap.getDigest())) {
+      map = dirtyMap;
+    } else {
+      // is it stored ?
+      UUID digest = store.fetchDigest(mapKey);
+      // not stored nor cached, nothing to do
+      if (digest == null) {
+        return false;
+      }
+      map = new PropertyMap(store, schemaOf(splitKey(mapName + ".*")), PropertyMap.DROPPED);
+      synchronized (dirtyMaps) {
+        dirtyMaps.put(mapName, map);
+      }
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapName, map);
+    }
+    return false;
+  }
+
+  /**
+   * Sets a property value.
+   * @param keys the key fragments
+   * @param value the new value or null if mapping should be removed
+   */
+  protected void setProperty(String[] keys, Object value) {
+    // find schema from key (length)
+    PropertySchema schema = schemaOf(keys);
+    String mapKey = mapKey(keys);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null) {
+      map = dirtyMap;
+    } else {
+      // is is stored ?
+      map = store.fetchProperties(mapKey, s->schema);
+      if (map == null) {
+        // remove a value from a non persisted map, noop
+        if (value == null) {
+          return;
+        }
+        map = new PropertyMap(store, schema);
+      }
+    }
+    // map is not null
+    String propertyName = propertyName(keys);
+    if (value != null) {
+      map.putProperty(propertyName, value);
+    } else {
+      map.removeProperty(propertyName);
+    }
+    // if this is the first update to the map
+    if (map != dirtyMap) {
+      dirtyMaps.put(mapKey, map);
+    }
+  }
+
+  /**
+   * Selects a set of properties.
+   * @param namePrefix the map name prefix
+   * @param predicateStr the condition selecting maps
+   * @param projectStr the projection property names or script
+   * @return the map of property maps keyed by their name
+   */
+  public Map<String, PropertyMap> selectProperties(String namePrefix, String predicateStr, String... projectStr) {
+    return selectProperties(namePrefix, predicateStr,
+        projectStr == null
+            ? Collections.emptyList()
+            : Arrays.asList(projectStr));
+  }
+
+  /**
+   * Selects a set of properties.
+   * @param namePrefix the map name prefix
+   * @param selector the selector/transformer function
+   * @return the map of property maps keyed by their name
+   */
+  public Map<String, PropertyMap> selectProperties(String namePrefix, Function<PropertyMap, PropertyMap> selector) {
+    final String mapKey = mapKey(namePrefix);
+    final Map<String, PropertyMap> selected = store.selectProperties(mapKey,null, k->schemaOf(splitKey(k)) );
+    final Map<String, PropertyMap> maps = new TreeMap<>();
+    final Function<PropertyMap, PropertyMap> transform = selector == null? Function.identity() : selector;
+    selected.forEach((k, p) -> {
+      final PropertyMap dirtyMap = dirtyMaps.get(k);

Review Comment:
   They are not in the case of a script where put properties can be followed by select in the same tx. Adding a test (and fixing a bug for new maps).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1209876978


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Digester.java:
##########
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hive.metastore.properties;
 
+import org.jetbrains.annotations.NotNull;

Review Comment:
   nit: looks like this import is none of use.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1203906794


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/Digester.java:
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.properties;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Date;
+import java.util.UUID;
+
+/**
+ * Helper class that creates a type 5 uuid.
+ * <p>This is computed from a set of updates using a SHA-1 message digest massaged into an UUID.
+ * see <a href="https://en.wikipedia.org/wiki/Universally_unique_identifier">...</a>
+ */
+public class Digester {
+    /** The Namespace uuid. */
+    private final UUID nsuid;
+    /** The digest used to compute the UUID. */
+    private final MessageDigest md;
+    /** A default namespace based on the class loading time. */
+    private static final UUID TEMP_NS;
+    static {
+        MessageDigest md = createDigest();
+        digest(md, System.currentTimeMillis());
+        TEMP_NS = computeUUID(md);
+    }
+
+    /**
+     * Allows to update the message digest from an object.
+     */
+    private static class TagOutputStream extends OutputStream {
+        /** The digest to update. */
+        private final MessageDigest md;
+
+        /**
+         * Sole ctor.
+         * @param md the message digester
+         */
+        TagOutputStream(MessageDigest md) {
+            this.md = md;
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            md.update((byte) b);
+        }
+
+        @Override
+        public void write(byte[] b, int off, int len) throws IOException {
+            md.update(b, off, len);
+        }
+    }
+
+    /**
+     * @return a SHA-1 message digest
+     */
+    private static MessageDigest createDigest() {
+        MessageDigest md;
+        try {
+            md = MessageDigest.getInstance("SHA-1");
+        } catch (NoSuchAlgorithmException nsae) {
+            throw new InternalError("SHA not supported");
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with a uuid.
+     * @param md the digest to update
+     * @param uid the uuid
+     */
+    private static MessageDigest digest(MessageDigest md, UUID uid) {
+        if (uid != null) {
+            long msb = uid.getMostSignificantBits();
+            digest(md, msb);
+            long lsb = uid.getLeastSignificantBits();
+            digest(md, lsb);
+        }
+        return md;
+    }
+
+    /**
+     * Updates a digest with an input stream.
+     * @param md the digest to update
+     * @param input the input to consume
+     * @throws IllegalStateException if an io exception occurs
+     */
+    private static void digest(MessageDigest md, InputStream input) {
+        try (OutputStream out = new TagOutputStream(md)) {
+            byte[] buffer = new byte[1024];
+            int read;
+            while ((read = input.read(buffer, 0, 1024)) >= 0) {
+                out.write(buffer, 0, read);
+            }
+        } catch(IOException xio) {
+            throw new IllegalStateException(xio);
+        }
+    }
+
+
+    /**
+     * Updates a digest with a long.
+     * @param md the digest to update
+     * @param l8 the long
+     */
+    private static void digest(MessageDigest md, long l8) {
+        md.update((byte) (l8 & 0xff));
+        md.update((byte) (l8 >> 8));
+        md.update((byte) (l8 >> 16));
+        md.update((byte) (l8 >> 24));
+        md.update((byte) (l8 >> 32));
+        md.update((byte) (l8 >> 40));
+        md.update((byte) (l8 >> 48));
+        md.update((byte) (l8 >> 56));
+    }
+
+    /**
+     * Updates a digest with an object.
+     * @param md the digest to update
+     * @param obj the object
+     */
+    private static void digest(MessageDigest md, Object obj) {
+        if (obj == null) {
+            return;
+        }
+        try (ObjectOutput out = new ObjectOutputStream(new TagOutputStream(md))) {
+            out.writeObject(obj);
+        } catch (IOException ex) {
+            // ignore close exception
+        }
+        // ignore close exception
+    }
+
+    /**
+     * Computes the uuid.
+     * @param md the message digest used to compute the hash
+     * @return the eTag as a type 5 uuid
+     */
+    private static UUID computeUUID(MessageDigest md) {
+        byte[] sha1Bytes = md.digest();
+        sha1Bytes[6] &= 0x0f;  /* clear version        */
+        sha1Bytes[6] |= 0x50;  /* set to version 5     */
+        sha1Bytes[8] &= 0x3f;  /* clear variant        */
+        sha1Bytes[8] |= 0x80;  /* set to IETF variant  */
+
+        // SHA generates 160 bytes; truncate to 128
+        long msb = 0;
+        //assert data.length == 16 || data.length == 20;
+        for (int i = 0; i < 8; i++) {
+            msb = (msb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        long lsb = 0;
+        for (int i = 8; i < 16; i++) {
+            lsb = (lsb << 8) | (sha1Bytes[i] & 0xff);
+        }
+        return new UUID(msb, lsb);
+    }
+
+    /**
+     * A marker interface for objects that can be digested.
+     */
+    public interface Digestible {
+        /**
+         * Updates a digest with this variable.
+         * @param digester the digester to update
+         * @return true if this digestible actually contributed to the digest
+         */
+        boolean digest(Digester digester);
+    }
+
+    /**
+     * A type 5 uuid is namespace + sha1; namespace in our case is a uuid.
+     * Two instances of digesters built with the same namespace will produce the same UUIDs from the
+     * same inputs.
+     * @param namespace the uuid namespace
+     */
+    public Digester(UUID namespace) {
+        nsuid = namespace == null? TEMP_NS : namespace;
+        md = createDigest();
+        // inject namespace
+        digest(md, nsuid);
+    }
+
+    /**
+     * A copy ctor base.
+     * @param lnsuid the namespace uid
+     * @param lmd the message digest
+     */
+    private Digester(UUID lnsuid, MessageDigest lmd)  {
+        this.nsuid = lnsuid;
+        this.md = lmd;
+    }
+
+    /**
+     * Default ctor.
+     * The created digester uses the class loading time as seed for its namespace; this means 2 instances of digester
+     * built in different JVM instances will *NOT* produce the same UUIDs for the same input. Typical use is in
+     * a non-persistent scenario, to verify an instance of an object has not been modified by checking
+     * its digested UUID remained the same.
+     * To get stable UUID computation across time and space in Digester usable in persistent scenario,
+     * you *NEED* to use a namespace-based digester using {@link Digester(UUID)}, uuid that is easily created
+     * using {@link UUID#nameUUIDFromBytes(byte[])} from any name/uri you might desire.
+     */
+    public Digester() {
+        this(null);
+    }
+
+    /**
+     * @return a clone of this instance
+     */
+    public Digester copy() {
+        try {
+            return new Digester(nsuid, (MessageDigest) md.clone());
+        } catch (CloneNotSupportedException ex) {
+            return null;
+        }
+    }
+
+    /**
+     * Computes the version tag from this digester.
+     * <p>This uses the current message digest state and resets it.
+     * @return the type 5 uuid
+     */
+    public UUID getUUID() {
+        UUID uuid = computeUUID(md);
+        md.reset();
+        digest(nsuid);
+        return uuid;
+    }
+
+    /**
+     * Updates the digest with a boolean.
+     * @param b the boolean
+     * @return this digester
+     */
+    public Digester digest(boolean b) {
+        md.update((byte) (b? 1 : 0));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a char.
+     * @param c the char
+     * @return this digester
+     */
+    public Digester digest(char c) {
+        md.update((byte) (c & 0xff));
+        md.update((byte) (c >> 8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a bytes array.
+     * @param bytes the bytes
+     * @return this digester
+     */
+    public Digester digest(byte[] bytes) {
+        if (bytes != null) {
+            md.update(bytes);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with an integer.
+     * @param i4 the int
+     * @return this digester
+     */
+    public Digester digest(int i4) {
+        md.update((byte) (i4 & 0xff));
+        md.update((byte) (i4 >> 8));
+        md.update((byte) (i4 >> 16));
+        md.update((byte) (i4 >> 24));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a long.
+     * @param l8 the long
+     * @return this digester
+     */
+    public Digester digest(long l8) {
+        digest(md, l8);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a double.
+     * @param f8 the double
+     * @return this digester
+     */
+    public Digester digest(double f8) {
+        digest(md, Double.doubleToRawLongBits(f8));
+        return this;
+    }
+
+    /**
+     * Updates the digest with a date.
+     * @param date the date
+     * @return this digester
+     */
+    public Digester digest(Date date) {
+        if (date != null) {
+            digest(md, date.getTime());
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a string.
+     * @param str the string
+     * @return this digester
+     */
+    public Digester digest(String str) {
+        if (str != null) {
+            final int sz = str.length();
+            for(int i = 0; i < sz; ++i) {
+                digest(str.charAt(i));
+            }
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uid the uuid
+     * @return this digester
+     */
+    public Digester digest(UUID uid) {
+        digest(md, uid);
+        return this;
+    }
+
+    /**
+     * Updates the digest with a uuid.
+     * @param uri the uri
+     * @return this digester
+     */
+    public Digester digest(URI uri) {
+        digest(md, uri.toString());
+        return this;
+    }
+
+    /**
+     * Updates the digest with an object that describes how it digests.
+     * @param digestible the object
+     * @return this digester
+     */
+    public Digester digest(Digestible digestible) {
+        if (digestible != null) {
+            digestible.digest(this);
+        }
+        return this;
+    }
+
+    /**
+     * Updates the digest with a stream.
+     * @param input the stream
+     * @return this digester
+     */
+    public Digester digest(InputStream input) {
+        if (input != null) {
+            digest(md, input);
+        }
+        return this;
+    }
+    /**
+     * Updates the digest with any (serializable) object.
+     * @param obj the object
+     * @return this digester
+     */
+    public Digester digest(Object obj) {
+        if (obj instanceof Digestible) {
+            return digest((Digestible) obj);
+        }
+        if (obj instanceof UUID) {
+            return digest((UUID) obj);
+        }
+        if (obj instanceof URI) {
+            return digest((URI) obj);
+        }
+        if (obj instanceof String) {
+            return digest((String) obj);
+        }
+        if (obj instanceof Date) {
+            return digest((Date) obj);
+        }
+        if (obj instanceof Integer) {
+            return digest(((Integer) obj).intValue());
+        }
+        if (obj instanceof Long) {
+            return digest(((Long) obj).longValue());
+        }
+        if (obj instanceof Double) {
+            return digest(((Double) obj).doubleValue());
+        }
+        if (obj instanceof Boolean) {
+            return digest(((Boolean) obj).booleanValue());
+        }
+        if (obj instanceof Character) {
+            return digest(((Character) obj).charValue());
+        }
+        if (obj instanceof Short) {
+            return digest(((Short) obj).intValue());
+        }
+        if (obj instanceof Float) {
+            return digest(((Float) obj).doubleValue());
+        }
+        if (obj != null && obj.getClass().isArray()) {
+           int sz  = java.lang.reflect.Array.getLength(obj);
+           for(int i = 0; i < sz; ++i) {
+               digest(java.lang.reflect.Array.get(obj, i));
+           }

Review Comment:
   nit: maybe we need to add `return this` after processing the `array`



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1203925764


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyStore.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.properties;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * The PropertyStore is the persistent container of property maps.
+ * Maps are addressed in the store by their key -  their name prepended by their manager&quot;s namespace.
+ */
+
+public abstract class PropertyStore {
+  /**
+   * Fetches a property map.
+   * @param mapKey the map key
+   * @param getSchema the method to retrieve a schema if the map needs to be created
+   * @return the item property map
+   */
+  public abstract PropertyMap fetchProperties(String mapKey, Function<String, PropertySchema> getSchema);
+
+
+  /**
+   * Fetches a map of property maps.
+   * @param keyPrefix the map key prefix
+   * @param keyFilter a filter for map keys
+   * @param getSchema the method to retrieve a schema if the map needs to be created
+   * @return the map of property map
+   */
+  public abstract Map<String, PropertyMap> selectProperties(final String keyPrefix, Predicate<String> keyFilter, Function<String, PropertySchema> getSchema);
+  /**
+   * Fetches a property map digest.
+   * @param mapKey the map key
+   * @return the item property map
+   */
+  public abstract UUID fetchDigest(String mapKey);
+
+  /**
+   * Fetches a map of property maps digest keyed by their name.
+   * @param keyPrefix the map key prefix
+   * @param keyFilter a filter for map keys
+   * @return the map of property map digests
+   */
+  public abstract Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter);
+
+    /**
+     * Persists a property map.
+     * @param mapKey the map key
+     * @param map the map instance
+     */
+  protected abstract void saveProperties(String mapKey, PropertyMap map);
+
+
+  /**
+   * Drops a property map.
+   * @param mapKey the map key
+   */
+  protected abstract boolean dropProperties(String mapKey);
+
+  /**
+   * Renames a property map.
+   * @param mapKey the map source key
+   * @param newKey the new target key
+   */
+  public abstract boolean renameProperties(String mapKey, String newKey);
+
+  /**
+   * Persists an iterator property map.
+   * <p>May be useful to override to use one transaction.</p>
+   * @param save the iterator on pairs for map key, property map
+   */
+  public void saveProperties(Iterator<Map.Entry<String, PropertyMap>> save) {
+    while(save.hasNext()) {
+      Map.Entry<String, PropertyMap> pair = save.next();
+      PropertyMap map = pair.getValue();
+      if (map != null) {
+        saveProperties(pair.getKey(), map);
+      } else {
+        dropProperties(pair.getKey());
+      }
+    }
+  }
+
+  /**
+   * Serializes a map as a byte array.
+   * @param map the (nonnull) map to write
+   * @return the byte array
+   */
+  public byte[] serialize(PropertyMap map) {
+    return SerializationProxy.toBytes(map);
+  }
+
+  /**
+   * Deserializes a map from a byte array.
+   * @param bytes the byte array
+   * @return the (nonnull) oroperty map
+   */
+  public PropertyMap deserialize(byte[] bytes, Function<String, PropertySchema> getSchema) {
+    return SerializationProxy.fromBytes(bytes, this, getSchema);
+  }
+
+  /**
+   * Default ctor.
+   */
+  protected PropertyStore() {
+  }
+
+  /**
+   * A non-persistent store, for tests mainly.
+   */
+  public static class Transient extends PropertyStore {

Review Comment:
   nit: If this is the class for test, could it move to the test package?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1204971322


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java:
##########
@@ -5841,9 +5851,7 @@ private <T> T doFetchProperties(String key, java.util.function.Function<MMetasto
         return (T) (transform != null? transform.apply(properties) : properties);
       }
     } finally {
-      if (query != null) {
-        query.closeAll();

Review Comment:
   nit: I'd prefer `query.closeAll()` other than `query.close()`, quoted from `Query` class
   
   ```
   Don't use this method directly; use closeAll() instead. It is intended for use with try-with-resources.
   Throws:
   Exception – if this resource cannot be closed
   ```
   If you want to use with try-with-resources, `QueryWrapper` is a choice.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 merged pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 merged PR #4194:
URL: https://github.com/apache/hive/pull/4194


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1158322429


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/JdoPropertyStore.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.commons.jexl3.JexlException;
+import org.apache.hadoop.hive.metastore.RawStore;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
+import org.apache.hadoop.hive.metastore.properties.PropertyException;
+import org.apache.hadoop.hive.metastore.properties.PropertyMap;
+import org.apache.hadoop.hive.metastore.properties.PropertySchema;
+import org.apache.hadoop.hive.metastore.properties.PropertyStore;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * Implementation of the property store delegating persistence to a (jdo) raw store.
+ */
+public class JdoPropertyStore extends PropertyStore {
+  /** The jdo objects store. */
+  private final RawStore objectStore;
+
+  /**
+   * Basic ctor.
+   * @param store the object store
+   */
+  public JdoPropertyStore(RawStore store) {
+    this.objectStore = store;
+  }
+
+  @Override
+  public PropertyMap fetchProperties(final String mapKey, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.getProperties(mapKey, getPropertyMapFunction(null, getSchema));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public Map<String, PropertyMap> selectProperties(final String keyPrefix, Predicate<String> keyFilter, Function<String, PropertySchema> getSchema) {
+    try {
+      return objectStore.selectProperties(keyPrefix, getPropertyMapFunction(keyFilter, getSchema));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public UUID fetchDigest(String mapKey) {
+    try {
+      return objectStore.getProperties(mapKey, (mm) -> UUID.fromString(mm.getPropertyValue()));
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public Map<String, UUID> selectDigest(String keyPrefix, Predicate<String> keyFilter) {
+    try {
+      return objectStore.selectProperties(keyPrefix, (mm) -> {
+        if (keyFilter == null || keyFilter.test(mm.getPropertykey())) {
+          return UUID.fromString(mm.getPropertyValue());
+        }
+        return null;
+      });
+    } catch (MetaException | JexlException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  public void saveProperties(Iterator<Map.Entry<String, PropertyMap>> save) {
+    // will run the super method in a transaction
+    try {
+      objectStore.runInTransaction(()-> super.saveProperties(save));
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override
+  protected void saveProperties(String mapKey, PropertyMap map) {
+    try {
+      if (map.isDropped()) {
+        objectStore.dropProperties(mapKey);
+      } else {
+        objectStore.putProperties(mapKey, map.getDigest().toString(), null, serialize(map));
+      }
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+
+  @Override public boolean dropProperties(String mapKey) {
+   try {
+     return objectStore.dropProperties(mapKey);
+   } catch (MetaException e) {
+     throw new PropertyException(e);
+   }
+  }
+
+  @Override public boolean renameProperties(String mapKey, String newKey) {
+    try {
+      return objectStore.renameProperties(mapKey, newKey);
+    } catch (MetaException e) {
+      throw new PropertyException(e);
+    }
+  }
+  /**
+   * Creates a function that transforms an MMetastoreDBProperties into a PropertyMap.
+   * @param keyFilter a map key filtering predicate that will make the function return null if test fails
+   * @param getSchema the function that solves a schema from a key
+   * @return a function
+   */
+  Function<MMetastoreDBProperties, PropertyMap> getPropertyMapFunction(final Predicate<String> keyFilter, final Function<String, PropertySchema> getSchema) {
+    return (mm) -> {
+      final String key = mm.getPropertykey();

Review Comment:
   Not necessary, just stating it won't change after init. In a method returning a lambda, I tried to be extra-clear.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1528697308

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [115 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1531508324

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [115 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4194: HIVE-27186: A persistent property store

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4194:
URL: https://github.com/apache/hive/pull/4194#issuecomment-1542380884

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4194)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG) [5 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY) [5 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4194&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL) [121 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4194&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4194&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174226987


##########
standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/properties/PocExternalTest.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.properties;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+
+/**
+ *
+ * @author henri
+ */
+public class PocExternalTest {

Review Comment:
   Same as above comment



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171284115


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();

Review Comment:
   why we need to cache the constructor for each namespace? seems there are only one implementation in production base



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171288113


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -75,22 +78,7 @@
 import java.lang.reflect.UndeclaredThrowableException;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
-import java.util.AbstractMap;

Review Comment:
   nit: can you restore the imports?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] saihemanth-cloudera commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "saihemanth-cloudera (via GitHub)" <gi...@apache.org>.
saihemanth-cloudera commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171939032


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/properties/PropertyManager.java:
##########
@@ -0,0 +1,576 @@
+/*
+ * 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.properties;
+
+
+import org.apache.commons.jexl3.JexlBuilder;
+import org.apache.commons.jexl3.JexlContext;
+import org.apache.commons.jexl3.JexlEngine;
+import org.apache.commons.jexl3.JexlException;
+import org.apache.commons.jexl3.JexlExpression;
+import org.apache.commons.jexl3.JexlFeatures;
+import org.apache.commons.jexl3.introspection.JexlPermissions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * A property manager.
+ * <p>
+ * This handles operations at the higher functional level; an instance is created per-session and
+ * drives queries and updates in a transactional manner.
+ * </p>
+ * <p>
+ * The manager ties the property schemas into one namespace; all property maps it handles must and will use
+ * one of its known schema.
+ * </p>
+ */
+public abstract class PropertyManager {
+  /** The logger. */
+  public static final Logger LOGGER = LoggerFactory.getLogger(PropertyManager.class);
+  /** The set of dirty maps. */
+  protected Map<String, PropertyMap> dirtyMaps = new HashMap<>();
+  /** This manager namespace. */
+  protected final String namespace;
+  /** The property map store. */
+  protected final PropertyStore store;
+  /** A Jexl engine for convenience. */
+  static final JexlEngine JEXL;
+  static {
+    JexlFeatures features = new JexlFeatures()
+        .sideEffect(false)
+        .sideEffectGlobal(false);
+    JexlPermissions p = JexlPermissions.RESTRICTED
+        .compose("org.apache.hadoop.hive.metastore.properties.*");
+    JEXL = new JexlBuilder()
+        .features(features)
+        .permissions(p)
+        .create();
+  }
+
+  /**
+   * The map of defined managers.
+   */
+  private static final Map<String, Constructor<? extends PropertyManager>> NSMANAGERS = new HashMap<>();
+
+  /**
+   * Declares a property manager class.
+   * @param ns the namespace
+   * @param pmClazz the property mamanger class
+   */
+  public static boolean declare(String ns, Class<? extends PropertyManager> pmClazz) {
+    try {
+      synchronized(NSMANAGERS) {
+        Constructor<? extends PropertyManager> ctor = NSMANAGERS.get(ns);
+        if (ctor == null) {
+          ctor = pmClazz.getConstructor(String.class, PropertyStore.class);
+          NSMANAGERS.put(ns, ctor);
+          return true;
+        } else {
+          if (!Objects.equals(ctor.getDeclaringClass(), pmClazz)) {
+            LOGGER.error("namespace {} is already declared for {}", ns, pmClazz.getCanonicalName());
+          }
+        }
+      }
+    } catch(NoSuchMethodException xnom ) {
+      LOGGER.error("namespace declaration failed: " + ns + ", " + pmClazz.getCanonicalName(),
+          xnom);
+    }
+    return false;
+  }
+
+  /**
+   * Creates an instance of manager using its declared namespace.
+   * @param namespace the manager&quot;s namespace
+   * @param store the property store
+   * @return a property manager instance
+   */
+  public static PropertyManager create(String namespace, PropertyStore store) {
+    final Constructor<? extends PropertyManager> ctor;
+    synchronized(NSMANAGERS) {
+      ctor = NSMANAGERS.get(namespace);
+    }
+    if (ctor != null) {
+      try {
+        return ctor.newInstance(namespace, store);
+      } catch(Exception xany) {
+        LOGGER.error("property manager creation failed "+ namespace, xany);
+      }
+    } else {
+      LOGGER.error("no such property manager namespace is declared " + namespace);
+    }
+    return null;
+  }
+
+  /**
+   * JEXL adapter.
+   */
+  public static class MapWrapper implements JexlContext {
+    PropertyMap map;
+    MapWrapper(PropertyMap map) {
+      this.map = map;
+    }
+
+    public Object get(String p) {
+      return map.getPropertyValue(p);
+    }
+
+    @Override
+    public void set(String name, Object value) {
+      map.putProperty(name, value);
+    }
+
+    @Override
+    public boolean has(String name) {
+      return map.getTypeOf(name) != null;
+    }
+  }
+
+  /**
+   * Creates a manager instance.
+   * @param store the store instance which must use an appropriate property map factory (probably use createMap).
+   */
+  protected PropertyManager(String ns, PropertyStore store) {
+    this.namespace = ns;
+    this.store = store;
+  }
+
+  /**
+   * Saves all pending updates to store.
+   */
+  public void commit() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      if (!dirtyMaps.isEmpty()) {
+        store.saveProperties(dirtyMaps.entrySet().iterator());
+        dirtyMaps.clear();
+      }
+    }
+  }
+
+  /**
+   * Forget all pending updates.
+   */
+  public void rollback() {
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      dirtyMaps.clear();
+    }
+  }
+
+  /**
+   * Imports a set of default values into this store&quot;s schema.
+   * The properties should be of the form schema_name.property_name=value.
+   * Note that this implies the manager has at least one known property map schema.
+   * @param importsp the properties
+   */
+  public void importDefaultValues(Properties importsp) {
+    importsp.forEach((k, v)->{
+      String importName = k.toString();
+      final int dotPosition = importName.indexOf(".");
+      if (dotPosition > 0) {
+        String schemaName = importName.substring(0, dotPosition);
+        PropertySchema schema = getSchema(schemaName);
+        if (schema != null) {
+          String propertyName = importName.substring(dotPosition + 1);
+          schema.setDefaultValue(propertyName, v);
+        }
+      }
+    });
+  }
+
+  /**
+   * Imports a set of property values.
+   * @param map the properties key=value
+   */
+  public void setProperties(Properties map) {
+    map.forEach((k, v)-> setProperty(k.toString(), v));
+  }
+
+  /**
+   * Injects a set of properties.
+   * If the value is null, the property is removed.
+   * @param map the map of properties to inject.
+   */
+  public void setProperties(Map<String, Object> map) {
+    map.forEach(this::setProperty);
+  }
+
+  /**
+   * Sets a property value.
+   * @param key the property key
+   * @param value the property value or null to unset
+   */
+  public void setProperty(String key, Object value) {
+    setProperty(splitKey(key), value);
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or null if not assigned
+   */
+  public Object getProperty(String key) {
+    return getProperty(splitKey(key));
+  }
+
+  /**
+   * Gets a property value.
+   * @param key the property key
+   * @return property value or the schema default value if not assigned
+   */
+  public Object getPropertyValue(String key) {
+    return getPropertyValue(splitKey(key));
+  }
+
+  /**
+   * Splits a property key into its fragments.
+   * @param key the property key
+   * @return the key fragments
+   */
+  protected String[] splitKey(String key) {
+    String[] splits = key.split("(?<!\\\\)\\.");
+    if (splits.length < 1) {
+      splits = new String[]{key};
+    }
+    return splits;
+  }
+
+  /**
+   * Gets a schema by name.
+   * <p>Only used by {@link #importDefaultValues(Properties)}</p>
+   * @param name schema name
+   * @return the schema instance, null if no such schema is known
+   */
+  public PropertySchema getSchema(String name) {
+    return null;
+  }
+
+  /**
+   * Determines the schema from the property key fragments.
+   * @param keys the key fragments
+   * @return the schema, {@link PropertySchema#NONE} if no such schema is known
+   */
+  protected PropertySchema schemaOf(String[] keys) {
+    return PropertySchema.NONE;
+  }
+
+  /**
+   * @param keys property key fragments
+   * @return number of fragments composing the map name in the fragments array
+   */
+  protected int getMapNameLength(String[] keys) {
+    return keys.length - 1;
+  }
+
+  /**
+   * Compose a property map key from a property map name.
+   * @param name the property map name, may be null or empty
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String name) {
+    StringBuilder strb = new StringBuilder(namespace);
+    if (name != null && !name.isEmpty()){
+      strb.append('.');
+      strb.append(name);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Extract a property map name from a property map key.
+   * @param key property map key
+   * @return the property map name
+   */
+  protected String mapName(String key) {
+    int dot = key.indexOf('.');
+    return dot > 0? key.substring(dot + 1) : key;
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the key fragments
+   * @return the property map key used by the store
+   */
+  protected String mapKey(String[] keys) {
+    return mapKey(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property map key from property key fragments.
+   * @param keys the property key fragments
+   * @param maxkl the maximum number of fragments in the map key
+   * @return the property key used by the store
+   */
+  protected String mapKey(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    // shortest map key is namespace
+    StringBuilder strb = new StringBuilder(namespace);
+    for(int k = 0; k < Math.min(maxkl, keys.length - 1); ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @return the property name
+   */
+  protected String propertyName(String[] keys) {
+    return propertyName(keys, getMapNameLength(keys));
+  }
+
+  /**
+   * Compose a property name from property key fragments.
+   * @param keys the key fragments
+   * @param maxkl the maximum number of fragments in the map name
+   * @return the property name
+   */
+  protected String propertyName(String[] keys, int maxkl) {
+    if (keys.length < 1) {
+      throw new IllegalArgumentException("at least 1 key fragments expected");
+    }
+    if (keys.length <= maxkl) {
+      return keys[keys.length - 1];
+    }
+    StringBuilder strb = new StringBuilder(keys[maxkl]);
+    for(int k = maxkl + 1; k < keys.length; ++k) {
+      strb.append('.');
+      strb.append(keys[k]);
+    }
+    return strb.toString();
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or null if none was assigned
+   */
+  public Object getProperty(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    if (map == null) {
+      map = store.fetchProperties(mapKey, null);
+    }
+    if (map != null) {
+      return map.getProperty(propertyName(keys));
+    }
+    return null;
+  }
+
+  /**
+   * Gets a property value.
+   * @param keys the key fragments
+   * @return the value or the default schema value if not assigned
+   */
+  public Object getPropertyValue(String[] keys) {
+    final String mapKey = mapKey(keys);
+    PropertyMap map;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized(dirtyMaps) {
+      map = dirtyMaps.get(mapKey);
+    }
+    PropertySchema schema = schemaOf(keys);
+    if (map == null) {
+      map = store.fetchProperties(mapKey, s->schema);
+    }
+    String propertyName = propertyName(keys);
+    if (map != null) {
+      return map.getPropertyValue(propertyName);
+    }
+    if (schema != null) {
+      return schema.getDefaultValue(propertyName);
+    }
+    return null;
+  }
+
+  /**
+   * Drops a property map.
+   * @param mapName the map name
+   * @return true if the properties may exist, false if they did nots
+   */
+  public boolean dropProperties(String mapName) {
+    final String mapKey = mapKey(mapName);
+    PropertyMap dirtyMap;
+    final Map<String, PropertyMap> dirtyMaps = this.dirtyMaps;
+    synchronized (dirtyMaps) {
+      dirtyMap = dirtyMaps.get(mapKey);
+    }
+    PropertyMap map;
+    if (dirtyMap != null && Objects.equals(PropertyMap.DROPPED, dirtyMap.getDigest())) {
+      map = dirtyMap;
+    } else {
+      // is is stored ?

Review Comment:
   Nit: is UUID stored?



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174272169


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -7535,6 +7523,38 @@ public List<Partition> get_partitions_by_names(final String dbName, final String
     return ret;
   }
 
+  /**
+   * Creates an instance of property manager based on the (declared) namespace.
+   * @param ns the namespace
+   * @return the manager instance
+   * @throws TException
+   */
+  private PropertyManager getPropertyManager(String ns) throws TException {
+    PropertyStore propertyStore = getMS().getPropertyStore();
+    PropertyManager mgr = PropertyManager.create(ns, propertyStore);
+    return mgr;
+  }
+  @Override
+  public PropertyGetResponse get_properties(PropertyGetRequest req) throws TException {
+    PropertyManager mgr = getPropertyManager(req.getNameSpace());
+    Map<String, PropertyMap> selected = mgr.selectProperties(req.getMapPrefix(), req.getMapPredicate(), req.getMapSelection());

Review Comment:
   I don't see how; the store, the dirty maps logic, the JEXL engine are members of that instance. I must be missing the point.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174279161


##########
standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/properties/PocExternalTest.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.properties;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+
+/**
+ *
+ * @author henri
+ */
+public class PocExternalTest {

Review Comment:
   Same too.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1174273842


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -2769,6 +2802,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   GetPartitionsByNamesResult get_partitions_by_names_req(1:GetPartitionsByNamesRequest req)
                         throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
+    // retrieve properties
+    PropertyGetResponse get_properties(1:PropertyGetRequest req);

Review Comment:
   Agreed.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] dengzhhu653 commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "dengzhhu653 (via GitHub)" <gi...@apache.org>.
dengzhhu653 commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1171297701


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -2769,6 +2802,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   GetPartitionsByNamesResult get_partitions_by_names_req(1:GetPartitionsByNamesRequest req)
                         throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
+    // retrieve properties
+    PropertyGetResponse get_properties(1:PropertyGetRequest req);

Review Comment:
   Should we declare the throwing exception of these two methods? I'm afraid the Thrift would throw transport exception instead when things go wrong.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175532386


##########
standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift:
##########
@@ -167,6 +167,39 @@ const string HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__"
 const string HIVE_FILTER_FIELD_TABLE_NAME = "hive_filter_field_tableName__"
 const string HIVE_FILTER_FIELD_TABLE_TYPE = "hive_filter_field_tableType__"
 
+/** Table maintenance operation type. */
+enum MaintenanceOpType {
+    COMPACTION = 1,
+    SNAPSHOT_EXPIRY = 2,
+    STATS_REBUILD= 3,
+    MV_BUILD= 4,

Review Comment:
   Removed them from Thrift def.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1176656863


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java:
##########
@@ -5665,6 +5678,212 @@ private String getGuidFromDB() throws MetaException {
     return null;
   }
 
+  @Override
+  public boolean runInTransaction(Runnable exec) throws MetaException {
+    boolean success = false;
+    Transaction tx = null;
+    try {
+      if (openTransaction()) {
+        exec.run();
+        success = commitTransaction();
+      }
+    } catch (Exception e) {
+      LOG.warn("Metastore operation failed", e);
+    } finally {
+      rollbackAndCleanup(success, null);
+    }
+    return success;
+  }
+
+  @Override
+  public boolean dropProperties(String key) throws MetaException {
+    boolean success = false;
+    Transaction tx = null;
+    Query query = null;
+    try {
+      if (openTransaction()) {
+        query = pm.newQuery(MMetastoreDBProperties.class, "this.propertyKey == key");
+        query.declareParameters("java.lang.String key");
+        Collection<MMetastoreDBProperties> properties = (Collection<MMetastoreDBProperties>) query.execute(key);
+        if (!properties.isEmpty()) {
+          pm.deletePersistentAll(properties);
+        }
+        success = commitTransaction();
+      }
+    } catch (Exception e) {
+      LOG.warn("Metastore property drop failed", e);
+    } finally {
+      rollbackAndCleanup(success, query);
+    }
+    return success;
+  }
+
+  @Override
+  public MMetastoreDBProperties putProperties(String key, String value, String description,  byte[] content) throws MetaException {
+    boolean success = false;
+    try {
+      if (openTransaction()) {
+        //pm.currentTransaction().setOptimistic(false);
+        // fetch first to determine new vs update
+        MMetastoreDBProperties properties = doGetProperties(key, null);
+        final boolean newInstance;
+        if (properties == null) {
+          newInstance = true;
+          properties = new MMetastoreDBProperties();
+          properties.setPropertykey(key);
+        } else {
+          newInstance = false;
+        }
+        properties.setDescription(description);
+        properties.setPropertyValue(value);
+        properties.setPropertyContent(content);
+        LOG.debug("Attempting to add property {} for the metastore db", key);
+        properties.setDescription("Metastore property "
+            + (newInstance ? "created" : "updated")
+            + " " + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")));
+        if (newInstance) {
+          pm.makePersistent(properties);
+        }
+        success = commitTransaction();
+        if (success) {
+          LOG.info("Metastore property {} created successfully", key);
+          return properties;
+        }
+      }
+    } catch (Exception e) {
+      LOG.warn("Metastore property save failed", e);
+    } finally {
+      rollbackAndCleanup(success, null);
+    }
+    return null;
+  }
+
+  @Override
+  public boolean renameProperties(String mapKey, String newKey) throws MetaException {
+    boolean success = false;
+    Transaction tx = null;
+    Query query = null;

Review Comment:
   query is cleaned up in finally block; QueryWrapper would not take care of the tx aspect.



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] henrib commented on a diff in pull request #4194: HIVE-27186: A persistent property store

Posted by "henrib (via GitHub)" <gi...@apache.org>.
henrib commented on code in PR #4194:
URL: https://github.com/apache/hive/pull/4194#discussion_r1175396173


##########
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/properties/HMSPropertyStoreRemoteTest.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.properties;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.jexl3.JxltEngine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HMSHandler;
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.JdoPropertyStore;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.ObjectStore;
+import org.apache.hadoop.hive.metastore.TestObjectStore;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MaintenanceOpStatus;
+import org.apache.hadoop.hive.metastore.api.MaintenanceOpType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.hive.metastore.properties.HMSPropertyManager.JEXL;
+import static org.apache.hadoop.hive.metastore.properties.HMSPropertyManager.MAINTENANCE_OPERATION;
+import static org.apache.hadoop.hive.metastore.properties.HMSPropertyManager.MAINTENANCE_STATUS;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.BOOLEAN;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.DATETIME;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.DOUBLE;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.INTEGER;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.JSON;
+import static org.apache.hadoop.hive.metastore.properties.PropertyType.STRING;
+
+public class HMSPropertyStoreRemoteTest extends HMSPropertyStoreTest {
+  //private static final String NS = "hms";
+  protected HiveMetaStoreClient client;
+  private boolean isServerStarted = false;
+  protected int port;
+
+  boolean createStore(Configuration conf, Warehouse wh) {
+    try {
+      MetaStoreTestUtils.setConfForStandloneMode(conf);
+      objectStore = new ObjectStore();
+      objectStore.setConf(conf);
+      //TestObjectStore.dropAllStoreObjects(objectStore);
+      HMSHandler.createDefaultCatalog(objectStore, wh);
+      // configure object store
+      objectStore.createDatabase(new DatabaseBuilder()
+          .setCatalogName("hive")
+          .setName(DB1)
+          .setDescription("description")
+          .setLocation("locationurl")
+          .build(conf));
+    } catch(InvalidObjectException | MetaException | InvalidOperationException xmeta) {
+      throw new PropertyException("unable to initialize server", xmeta);
+    }
+    return true;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
+    // Events that get cleaned happen in batches of 1 to exercise batching code
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.EVENT_CLEAN_MAX_EVENTS, 1L);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    if (isServerStarted) {
+      Assert.assertNotNull("Unable to connect to the MetaStore server", client);
+      return;
+    }
+    port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    System.out.println("Starting MetaStore Server on port " + port);
+    isServerStarted = true;
+
+    Warehouse wh = new Warehouse(conf);
+    boolean inited = createStore(conf, wh);
+    LOG.info("MetaStore Thrift Server test initialization " + (inited? "successful":"failed"));
+    // This is default case with setugi off for both client and server
+    client = createClient();
+  }
+
+
+  protected HiveMetaStoreClient createClient() throws Exception {
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.EXECUTE_SET_UGI, false);
+    HiveMetaStoreClient client = new HiveMetaStoreClient(conf);
+    return client;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    MetaStoreTestUtils.close(port);
+    super.tearDown();
+  }
+
+  @Test
+  public void testHMSProperties() throws Exception {

Review Comment:
   Refactored the unit tests (and added a Servlet skeleton to test another transport).



-- 
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: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org