You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2020/03/02 15:00:40 UTC

[drill] 03/08: DRILL-7547: Support credentials store for mongo connections

This is an automated email from the ASF dual-hosted git repository.

arina pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 0c5e347341830e86a64491dd0cd6bf2f723e0072
Author: Dobes Vandermeer <do...@gmail.com>
AuthorDate: Thu Feb 27 17:41:07 2020 -0800

    DRILL-7547: Support credentials store for mongo connections
    
    This uses the hadoop `Configuration.getPassword` method to retrieve the
    username and password for mongo connections.  This allows the user to
    supply credentials or credential store configuration in core-site.xml
    instead of inlining the credentials in the storage plugin configuration
    that is stored in ZooKeeper.
    
    Refer to the CredentialProviderAPI document for more information about
    how credential provider plugins work.
    
    closes #2001
---
 .../drill/exec/store/mongo/MongoStoragePlugin.java |  79 +++++++++------
 .../drill/exec/store/mongo/MongoTestBase.java      |   6 +-
 .../{MongoTestSuit.java => MongoTestSuite.java}    |  52 +++++-----
 ...TestMongoStoragePluginUsesCredentialsStore.java |  89 +++++++++++++++++
 .../storage-mongo/src/test/resources/core-site.xml |  37 ++++---
 .../src/main/resources/core-site-example.xml       | 108 ++++++++++++++++++---
 6 files changed, 287 insertions(+), 84 deletions(-)

diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoStoragePlugin.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoStoragePlugin.java
index 93a80e6..2dfba8c 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoStoragePlugin.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoStoragePlugin.java
@@ -17,13 +17,12 @@
  */
 package org.apache.drill.exec.store.mongo;
 
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.mongodb.MongoClient;
+import com.mongodb.MongoClientURI;
+import com.mongodb.MongoCredential;
+import com.mongodb.ServerAddress;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -34,43 +33,68 @@ import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 import org.apache.drill.exec.store.mongo.schema.MongoSchemaFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.shaded.guava.com.google.common.cache.Cache;
 import org.apache.drill.shaded.guava.com.google.common.cache.CacheBuilder;
 import org.apache.drill.shaded.guava.com.google.common.cache.RemovalListener;
 import org.apache.drill.shaded.guava.com.google.common.cache.RemovalNotification;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import com.mongodb.MongoClient;
-import com.mongodb.MongoClientURI;
-import com.mongodb.MongoCredential;
-import com.mongodb.ServerAddress;
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URLEncoder;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 public class MongoStoragePlugin extends AbstractStoragePlugin {
-  static final Logger logger = LoggerFactory
-      .getLogger(MongoStoragePlugin.class);
+  private static final Logger logger = LoggerFactory.getLogger(MongoStoragePlugin.class);
 
   private final MongoStoragePluginConfig mongoConfig;
   private final MongoSchemaFactory schemaFactory;
   private final Cache<MongoCnxnKey, MongoClient> addressClientMap;
   private final MongoClientURI clientURI;
 
-  public MongoStoragePlugin(MongoStoragePluginConfig mongoConfig,
-      DrillbitContext context, String name) throws IOException,
-      ExecutionSetupException {
+  public MongoStoragePlugin(
+    MongoStoragePluginConfig mongoConfig,
+    DrillbitContext context,
+    String name) throws IOException, ExecutionSetupException {
     super(context, name);
     this.mongoConfig = mongoConfig;
-    this.clientURI = new MongoClientURI(this.mongoConfig.getConnection());
+    String connection = addCredentialsFromCredentialsProvider(this.mongoConfig.getConnection(), name);
+    this.clientURI = new MongoClientURI(connection);
     this.addressClientMap = CacheBuilder.newBuilder()
-        .expireAfterAccess(24, TimeUnit.HOURS)
-        .removalListener(new AddressCloser()).build();
+      .expireAfterAccess(24, TimeUnit.HOURS)
+      .removalListener(new AddressCloser()).build();
     this.schemaFactory = new MongoSchemaFactory(this, name);
   }
 
+  private static final String addCredentialsFromCredentialsProvider(String connection, String name) {
+    MongoClientURI parsed = new MongoClientURI(connection);
+    if (parsed.getCredentials() == null) {
+      Configuration configuration = new Configuration();
+      try {
+        // The default connection has the name "mongo" but multiple connections can be added;
+        // each will need their own credentials.
+        char[] usernameChars = configuration.getPassword("drill.exec.store." + name + ".username");
+        char[] passwordChars = configuration.getPassword("drill.exec.store." + name + ".password");
+        if (usernameChars != null && passwordChars != null) {
+          String username = URLEncoder.encode(new String(usernameChars), "UTF-8");
+          String password = URLEncoder.encode(new String(passwordChars), "UTF-8");
+          String updatedUrl = connection.replaceFirst("://", "://" + username + ":" + password + "@");
+          return updatedUrl.toString();
+        }
+      } catch (IOException e) {
+        logger.error("Error fetching mongodb username and password from configuration", e);
+      }
+    }
+    return connection;
+  }
+
   @Override
   public MongoStoragePluginConfig getConfig() {
     return mongoConfig;
@@ -88,7 +112,8 @@ public class MongoStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
-    MongoScanSpec mongoScanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<MongoScanSpec>() {});
+    MongoScanSpec mongoScanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<MongoScanSpec>() {
+    });
     return new MongoGroupScan(userName, this, mongoScanSpec, null);
   }
 
@@ -99,10 +124,10 @@ public class MongoStoragePlugin extends AbstractStoragePlugin {
 
 
   private class AddressCloser implements
-      RemovalListener<MongoCnxnKey, MongoClient> {
+    RemovalListener<MongoCnxnKey, MongoClient> {
     @Override
     public synchronized void onRemoval(
-        RemovalNotification<MongoCnxnKey, MongoClient> removal) {
+      RemovalNotification<MongoCnxnKey, MongoClient> removal) {
       removal.getValue().close();
       logger.debug("Closed connection to {}.", removal.getKey().toString());
     }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
index aff0b56..f1b4e4d 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
@@ -32,8 +32,8 @@ public class MongoTestBase extends ClusterTest implements MongoTestConstants {
     startCluster(ClusterFixture.builder(dirTestWatcher));
     pluginRegistry = cluster.drillbit().getContext().getStorage();
 
-    MongoTestSuit.initMongo();
-    initMongoStoragePlugin(MongoTestSuit.getConnectionURL());
+    MongoTestSuite.initMongo();
+    initMongoStoragePlugin(MongoTestSuite.getConnectionURL());
   }
 
   private static void initMongoStoragePlugin(String connectionURI) throws Exception {
@@ -52,7 +52,7 @@ public class MongoTestBase extends ClusterTest implements MongoTestConstants {
   @AfterClass
   public static void tearDownMongoTestBase() throws Exception {
     pluginRegistry.deletePlugin(MongoStoragePluginConfig.NAME);
-    MongoTestSuit.tearDownCluster();
+    MongoTestSuite.tearDownCluster();
   }
 
 }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuite.java
similarity index 98%
rename from contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
rename to contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuite.java
index 9dce09d..6f8b165 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuite.java
@@ -17,35 +17,12 @@
  */
 package org.apache.drill.exec.store.mongo;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.drill.categories.MongoStorageTest;
-import org.apache.drill.categories.SlowTest;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import org.apache.drill.test.BaseTest;
-import org.bson.Document;
-import org.bson.conversions.Bson;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-import org.junit.runners.Suite.SuiteClasses;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import com.mongodb.MongoClient;
 import com.mongodb.ServerAddress;
 import com.mongodb.client.MongoCollection;
 import com.mongodb.client.MongoDatabase;
 import com.mongodb.client.model.IndexOptions;
 import com.mongodb.client.model.Indexes;
-
 import de.flapdoodle.embed.mongo.Command;
 import de.flapdoodle.embed.mongo.MongodExecutable;
 import de.flapdoodle.embed.mongo.MongodProcess;
@@ -63,14 +40,35 @@ import de.flapdoodle.embed.mongo.distribution.Version;
 import de.flapdoodle.embed.mongo.tests.MongosSystemForTestFactory;
 import de.flapdoodle.embed.process.config.IRuntimeConfig;
 import de.flapdoodle.embed.process.runtime.Network;
+import org.apache.drill.categories.MongoStorageTest;
+import org.apache.drill.categories.SlowTest;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.BaseTest;
+import org.bson.Document;
+import org.bson.conversions.Bson;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+import org.junit.runners.Suite.SuiteClasses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 
 @RunWith(Suite.class)
 @SuiteClasses({ TestMongoFilterPushDown.class, TestMongoProjectPushDown.class,
     TestMongoQueries.class, TestMongoChunkAssignment.class })
 @Category({SlowTest.class, MongoStorageTest.class})
-public class MongoTestSuit extends BaseTest implements MongoTestConstants {
+public class MongoTestSuite extends BaseTest implements MongoTestConstants {
 
-  private static final Logger logger = LoggerFactory.getLogger(MongoTestSuit.class);
+  private static final Logger logger = LoggerFactory.getLogger(MongoTestSuite.class);
   protected static MongoClient mongoClient;
 
   private static boolean distMode = Boolean.valueOf(System.getProperty("drill.mongo.tests.shardMode", "false"));
@@ -242,7 +240,7 @@ public class MongoTestSuit extends BaseTest implements MongoTestConstants {
 
   @BeforeClass
   public static void initMongo() throws Exception {
-    synchronized (MongoTestSuit.class) {
+    synchronized (MongoTestSuite.class) {
       if (initCount.get() == 0) {
         if (distMode) {
           logger.info("Executing tests in distributed mode");
@@ -282,7 +280,7 @@ public class MongoTestSuit extends BaseTest implements MongoTestConstants {
 
   @AfterClass
   public static void tearDownCluster() throws Exception {
-    synchronized (MongoTestSuit.class) {
+    synchronized (MongoTestSuite.class) {
       if (initCount.decrementAndGet() == 0) {
         try {
           if (mongoClient != null) {
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoStoragePluginUsesCredentialsStore.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoStoragePluginUsesCredentialsStore.java
new file mode 100644
index 0000000..58d9bf2
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoStoragePluginUsesCredentialsStore.java
@@ -0,0 +1,89 @@
+/*
+ * 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.drill.exec.store.mongo;
+
+import com.mongodb.MongoCredential;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestMongoStoragePluginUsesCredentialsStore {
+
+  private void test(String expectedUserName, String expectedPassword, String connection, String name) throws IOException, ExecutionSetupException {
+    MongoStoragePlugin plugin = new MongoStoragePlugin(new MongoStoragePluginConfig(
+      connection), null, name);
+    List<MongoCredential> creds = plugin.getClient().getCredentialsList();
+    if (expectedUserName == null) {
+      assertEquals(0, creds.size());
+    } else {
+      assertEquals(1, creds.size());
+      MongoCredential cred = creds.get(0);
+      assertEquals(expectedUserName, cred.getUserName());
+      assertEquals(expectedPassword, new String(cred.getPassword()));
+    }
+  }
+
+  @Test
+  public void usesPasswordFromCoreSiteXml() throws Exception {
+    test(
+      "test",
+      "pw",
+      "mongodb://example:27017/somedb?readPreference=primary",
+      "mongo");
+  }
+
+  @Test
+  public void supportsInterpolation() throws Exception {
+    test(
+      "mooch_test",
+      "mooch_pw",
+      "mongodb://example:27017/somedb?readPreference=primary",
+      "mongomooch");
+  }
+
+  @Test
+  public void doesNotReplaceExistingCredentials() throws Exception {
+    test(
+      "u",
+      "p",
+      "mongodb://u:p@example:27017/somedb?readPreference=primary",
+      "mongo");
+  }
+
+  @Test
+  public void urlEncodesCredentials() throws Exception {
+    test(
+      "!@#$%//:+^*()",
+      ":;[]}{!@#/?$#@",
+      "mongodb://example:27017/somedb?readPreference=primary",
+      "mongosec");
+  }
+
+  @Test
+  public void doesNotAddCredentialsIfNoneFound() throws Exception {
+    test(
+      null,
+      null,
+      "mongodb://example:27017/somedb?readPreference=primary",
+      "mongopublic");
+  }
+}
diff --git a/distribution/src/main/resources/core-site-example.xml b/contrib/storage-mongo/src/test/resources/core-site.xml
similarity index 55%
copy from distribution/src/main/resources/core-site-example.xml
copy to contrib/storage-mongo/src/test/resources/core-site.xml
index c7225a1..43b6863 100644
--- a/distribution/src/main/resources/core-site-example.xml
+++ b/contrib/storage-mongo/src/test/resources/core-site.xml
@@ -21,27 +21,32 @@
 <configuration>
 
     <property>
-        <name>fs.s3a.access.key</name>
-        <value>ENTER_YOUR_ACCESSKEY</value>
+        <name>drill.exec.store.mongo.username</name>
+        <value>test</value>
     </property>
 
     <property>
-        <name>fs.s3a.secret.key</name>
-        <value>ENTER_YOUR_SECRETKEY</value>
+        <name>drill.exec.store.mongo.password</name>
+        <value>pw</value>
     </property>
 
-    <!--Use this property to specify one or more credential provider URIs instead of
-        configuring above credentials in plain text-->
-    <!--<property>
-        <name>hadoop.security.credential.provider.path</name>
-        <value>ENTER_YOUR_PROVIDER_PATH</value>
-    </property>-->
+    <property>
+        <name>drill.exec.store.mongosec.username</name>
+        <value>!@#$%//:+^*()</value>
+    </property>
 
-    <!--Set this property to true to avoid caching of S3 file system configuration properties,
-        so when you add/update a property (e.g. fs.s3a.secret.key) in S3 storage plugin its new value will be taken-->
-    <!--<property>
-        <name>fs.s3a.impl.disable.cache</name>
-        <value>true</value>
-    </property>-->
+    <property>
+        <name>drill.exec.store.mongosec.password</name>
+        <value>:;[]}{!@#/?$#@</value>
+    </property>
 
+    <property>
+        <name>drill.exec.store.mongomooch.username</name>
+        <value>mooch_${drill.exec.store.mongo.username}</value>
+    </property>
+
+    <property>
+        <name>drill.exec.store.mongomooch.password</name>
+        <value>mooch_${drill.exec.store.mongo.password}</value>
+    </property>
 </configuration>
diff --git a/distribution/src/main/resources/core-site-example.xml b/distribution/src/main/resources/core-site-example.xml
index c7225a1..3ce2e29 100644
--- a/distribution/src/main/resources/core-site-example.xml
+++ b/distribution/src/main/resources/core-site-example.xml
@@ -19,29 +19,115 @@
 
 -->
 <configuration>
+    <!--
+      Note that values can make use of environment variables by using variable
+      substitution, e.g. ${env.AWS_ACCESS_KEY_ID} will be replaced with the
+      AWS_ACCESS_KEY_ID environment variable.
 
+      If you are deploying in a containerized environment this can be handy
+      since you can have all the settings in here use environment variables
+      and set those variables appropriately for each environment using
+      environment variables, which are often easier to modify individually
+      than a config file.
+    -->
+    <!--
+      Credentials for the AWS S3 storage plugin can be set here using
+      fs.s3a.access.key and fs.s3a.secret.key properties.
+    -->
+    <!--
     <property>
         <name>fs.s3a.access.key</name>
-        <value>ENTER_YOUR_ACCESSKEY</value>
+        <value>${env.AWS_ACCESS_KEY_ID}</value>
     </property>
 
     <property>
         <name>fs.s3a.secret.key</name>
-        <value>ENTER_YOUR_SECRETKEY</value>
+        <value>${env.AWS_SECRET_ACCESS_KEY}</value>
     </property>
+    -->
 
-    <!--Use this property to specify one or more credential provider URIs instead of
-        configuring above credentials in plain text-->
-    <!--<property>
+    <!--
+      Credentials for the mongo storage plugin can be provided here instead of in the plugin configuration.  Note
+      that to configure a plugin with the name "foo" you would set drill.exec.store.foo.username; the examples here
+      use the default plugin name of "mongo".
+    -->
+    <!--
+    <property>
+        <name>drill.exec.store.mongo.username</name>
+        <value>${env.MONGO_USERNAME}</value>
+        <description>
+           Username for mongo storage plugin
+        </description>
+    </property>
+
+    <property>
+        <name>drill.exec.store.mongo.password</name>
+        <value>${env.MONGO_PASSWORD}</value>
+        <description>
+          Password for mongo storage plugin.
+        </description>
+    </property>
+    -->
+
+    <!-- Credentials Provider Configuration -->
+    <!--
+    <property>
         <name>hadoop.security.credential.provider.path</name>
-        <value>ENTER_YOUR_PROVIDER_PATH</value>
-    </property>-->
+        <value>jceks://file/opt/drill/conf/credentials.jceks</value>
+        <description>
+          Use this property to specify one or more credential provider URIs
+          instead of configuring credentials in plaintext using the properties above.
+
+          To store credentials in the filesystem you can use a value like:
+
+          jceks://file/opt/drill/conf/credentials.jceks
+
+          You can also store credentials in S3 (except the S3 credentials themselves):
+
+          jceks://s3a@drill-credentials/credentials.jceks
+
+          You may specify multiple provider paths, comma-separated:
 
-    <!--Set this property to true to avoid caching of S3 file system configuration properties,
-        so when you add/update a property (e.g. fs.s3a.secret.key) in S3 storage plugin its new value will be taken-->
-    <!--<property>
+          jceks://file/opt/drill/conf/credentials.jceks,jceks://s3a@drill-credentials/credentials.jceks
+
+          To create/set credentials, set this property, install Hadoop, and use the hadoop CLI:
+
+          export HADOOP_CONF_DIR=/opt/drill/conf
+          hadoop credential create fs.s3a.access.key
+          hadoop credential create fs.s3a.access.secret
+          hadoop credential create drill.exec.store.mongo.username -provider jceks://s3a@creds/mongo.jceks
+          hadoop credential create drill.exec.store.mongo.password -provider jceks://s3a@creds/mongo.jceks
+
+          Refer to the CredentialProviderAPI documentation for more details:
+
+          https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/CredentialProviderAPI.html
+        </description>
+    </property>
+
+    <property>
+      <name>hadoop.security.credstore.java-keystore-provider.password-file</name>
+      <value>/opt/drill/conf/credentials-password.txt</value>
+      <description>
+        You may set this property to the name of a file containing a secret string that will be used to
+        encrypt / decrypt credentials.  If this is not specified, a default password of "none" will be used.
+
+        This may provide added security if the credentials are stored in a different filesystem from the
+        password file itself. For example, if the credentials file were stored on S3 and an attacker accessed
+        it, they would still need access to this password file before they could make use of the credentials.
+      </description>
+    </property>
+    -->
+
+    <!--
+      Set this property to true to avoid caching of S3 file system configuration properties,
+      so when you add/update a property (e.g. fs.s3a.secret.key) in S3 storage plugin its new
+      value will be taken
+    -->
+    <!--
+    <property>
         <name>fs.s3a.impl.disable.cache</name>
         <value>true</value>
-    </property>-->
+    </property>
+    -->
 
 </configuration>