You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2017/09/08 19:06:55 UTC

phoenix git commit: PHOENIX-4150 Adding property policy for user defined properties

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 016c0a976 -> 4959e88e8


PHOENIX-4150 Adding property policy for user defined properties

Signed-off-by: Thomas D'Silva <td...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/4959e88e
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/4959e88e
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/4959e88e

Branch: refs/heads/4.x-HBase-0.98
Commit: 4959e88e8821a6cd260829d367718cc08c76a34b
Parents: 016c0a9
Author: aertoria <ca...@gmail.com>
Authored: Mon Sep 4 12:31:13 2017 -0700
Committer: Thomas D'Silva <td...@apache.org>
Committed: Fri Sep 8 12:06:15 2017 -0700

----------------------------------------------------------------------
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  5 ++
 .../query/PropertyNotAllowedException.java      | 19 ++++++
 .../apache/phoenix/query/PropertyPolicy.java    | 61 ++++++++++++++++++++
 .../phoenix/query/PropertyPolicyProvider.java   | 39 +++++++++++++
 .../query/PropertyPolicyProviderTest.java       | 52 +++++++++++++++++
 .../phoenix/query/TestPropertyPolicy.java       | 34 +++++++++++
 .../org.apache.phoenix.query.PropertyPolicy     |  1 +
 7 files changed, 211 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4959e88e/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 6d2dd72..3896374 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -79,6 +79,7 @@ import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.ConnectionQueryServices.Feature;
 import org.apache.phoenix.query.DelegateConnectionQueryServices;
 import org.apache.phoenix.query.MetaDataMutated;
+import org.apache.phoenix.query.PropertyPolicyProvider;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -217,6 +218,10 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
         GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER.increment();
         this.url = url;
         this.isDescVarLengthRowKeyUpgrade = isDescVarLengthRowKeyUpgrade;
+
+        // Filter user provided properties based on property policy, if provided.
+        PropertyPolicyProvider.getPropertyPolicy().evaluate(info);
+
         // Copy so client cannot change
         this.info = info == null ? new Properties() : PropertiesUtil.deepCopy(info);
         final PName tenantId = JDBCUtil.getTenantId(url, info);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4959e88e/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyNotAllowedException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyNotAllowedException.java b/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyNotAllowedException.java
new file mode 100644
index 0000000..ed0e23e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyNotAllowedException.java
@@ -0,0 +1,19 @@
+package org.apache.phoenix.query;
+
+import java.sql.SQLException;
+import java.util.Properties;
+
+/**
+ * Exception thrown when one or more properties that is not allowed by {@link PropertyPolicy}
+ */
+public class PropertyNotAllowedException extends SQLException {
+  private static final long serialVersionUID = 1L;
+  private final Properties offendingProperties;
+
+  public PropertyNotAllowedException(Properties offendingProperties){
+    this.offendingProperties=offendingProperties;
+  }
+
+  public Properties getOffendingProperties(){ return this.offendingProperties; }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4959e88e/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyPolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyPolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyPolicy.java
new file mode 100644
index 0000000..3314a13
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyPolicy.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.query;
+import java.util.Properties;
+
+/**
+ * Filters {@link Properties} instances based on property policy.
+ *
+ * Provided properties, check if each property inside is whitelisted,
+ * if not, throws IllegalArgumentException.
+ * For best code practice, throws the offending properties list along
+ * with exception
+ *
+ * An example will be:
+ *<code>
+ *public class Customized PropertyPolicy implements PropertyPolicy {
+ *  final static Set<String> propertiesKeyAllowed = Collections.unmodifiableSet(
+ *      new HashSet<>(asList("DriverMajorVersion","DriverMinorVersion","DriverName","CurrentSCN")));
+ *
+ *      @Override public void evaluate(Properties properties) {
+ *      final Set<String> offendingProperties = new HashSet<>();
+ *
+ *      for(Object k:properties.keySet()){
+ *          if (propertiesKeyDisAllowed.contains(k)) offendingProperties.put((String)k,properties.getProperty((String)k));
+ *      }
+ *
+ *      if (offendingProperties.size()>0) throw new IllegalArgumentException("properties not allowed. offending properties" + offendingProperties);
+ *  }
+ *}
+ *</code>
+ */
+public interface PropertyPolicy {
+    /**
+     * @param properties
+     * @throws IllegalArgumentException
+     */
+    void evaluate(Properties properties) throws PropertyNotAllowedException;
+
+    /**
+     * Default implementation allows all properties.
+     */
+    static class PropertyPolicyImpl implements PropertyPolicy {
+        @Override
+        public void evaluate(Properties properties) throws PropertyNotAllowedException{}
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4959e88e/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyPolicyProvider.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyPolicyProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyPolicyProvider.java
new file mode 100644
index 0000000..27cb2d3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/PropertyPolicyProvider.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.query;
+
+import org.apache.phoenix.util.InstanceResolver;
+
+/**
+ * Manages providers that provide property policy for Phoenix.
+ * <p/>
+ * Dependent modules may register their own implementations of the following using {@link java.util.ServiceLoader}:
+ * <ul>
+ *     <li>{@link PropertyPolicy}</li>
+ * </ul>
+ *
+ * If a custom implementation is not registered, the default implementations will be used.
+ *
+ */
+public class PropertyPolicyProvider {
+    private static final PropertyPolicy DEFAULT_PROPERTY_POLICY = new PropertyPolicy.PropertyPolicyImpl();
+
+    public static PropertyPolicy getPropertyPolicy(){
+        return InstanceResolver.getSingleton(PropertyPolicy.class, DEFAULT_PROPERTY_POLICY);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4959e88e/phoenix-core/src/test/java/org/apache/phoenix/query/PropertyPolicyProviderTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/PropertyPolicyProviderTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/PropertyPolicyProviderTest.java
new file mode 100644
index 0000000..053448e
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/PropertyPolicyProviderTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.phoenix.query;
+
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.junit.Assert.assertTrue;
+
+public class PropertyPolicyProviderTest extends BaseConnectionlessQueryTest{
+    @Test
+    public void testPropertyPolicyProvider() {
+        PropertyPolicy provided = PropertyPolicyProvider.getPropertyPolicy();
+        assertTrue(provided instanceof TestPropertyPolicy);
+    }
+
+    @Test(expected = PropertyNotAllowedException.class)
+    public void testPropertyPolicyBlacklisted() throws SQLException {
+        Properties properties=new Properties();
+        properties.put("DisallowedProperty","value");
+        try(Connection conn = DriverManager.getConnection(getUrl(),properties);
+        ){}
+    }
+
+    @Test
+    public void testPropertyPolicyWhitelisted() throws SQLException {
+        Properties properties=new Properties();
+        properties.put("allowedProperty","value");
+        try(
+        Connection conn = DriverManager.getConnection(getUrl(),properties);
+        ){}
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4959e88e/phoenix-core/src/test/java/org/apache/phoenix/query/TestPropertyPolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/TestPropertyPolicy.java b/phoenix-core/src/test/java/org/apache/phoenix/query/TestPropertyPolicy.java
new file mode 100644
index 0000000..ddc452c
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/TestPropertyPolicy.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright, 1999-2013, SALESFORCE.com
+ * All Rights Reserved
+ * Company Confidential
+ */
+package org.apache.phoenix.query;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import static java.util.Arrays.asList;
+
+/**
+ * Configuration factory that populates an {@link Configuration} with static and runtime
+ * configuration settings.
+ */
+public class TestPropertyPolicy implements PropertyPolicy {
+  final static Set<String> propertiesKeyDisAllowed = Collections.unmodifiableSet(
+      new HashSet<>(asList("DisallowedProperty")));
+
+  @Override public void evaluate(Properties properties) throws PropertyNotAllowedException {
+    final Properties offendingProperties = new Properties();
+
+    for(Object k:properties.keySet()){
+      if (propertiesKeyDisAllowed.contains(k)) offendingProperties.put((String)k,properties.getProperty((String)k));
+    }
+
+    if (offendingProperties.size()>0) throw new PropertyNotAllowedException(offendingProperties);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4959e88e/phoenix-core/src/test/resources/META-INF/services/org.apache.phoenix.query.PropertyPolicy
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/resources/META-INF/services/org.apache.phoenix.query.PropertyPolicy b/phoenix-core/src/test/resources/META-INF/services/org.apache.phoenix.query.PropertyPolicy
new file mode 100644
index 0000000..07960b6
--- /dev/null
+++ b/phoenix-core/src/test/resources/META-INF/services/org.apache.phoenix.query.PropertyPolicy
@@ -0,0 +1 @@
+org.apache.phoenix.query.TestPropertyPolicy
\ No newline at end of file