gora-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From lewismc <...@git.apache.org>
Subject [GitHub] gora pull request #110: Initial commit gora cassandra cql
Date Mon, 19 Jun 2017 20:19:04 GMT
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/110#discussion_r122809493
  
    --- Diff: gora-cassandra-cql/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java
---
    @@ -0,0 +1,516 @@
    +/*
    + *  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.gora.cassandra.store;
    +
    +import com.datastax.driver.core.*;
    +import com.datastax.driver.core.policies.*;
    +import org.apache.gora.persistency.BeanFactory;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.PartitionQuery;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.Result;
    +import org.apache.gora.store.impl.DataStoreBase;
    +import org.apache.hadoop.conf.Configuration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.DataInput;
    +import java.io.DataOutput;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Properties;
    +
    +public class CassandraStore<K, T extends PersistentBase> extends DataStoreBase<K,
T> {
    +
    +  /**
    +   * Consistency property level for Cassandra column families
    +   */
    +  private static final String COL_FAM_CL = "cf.consistency.level";
    +
    +  /**
    +   * Consistency property level for Cassandra read operations.
    +   */
    +  private static final String READ_OP_CL = "read.consistency.level";
    +
    +  /**
    +   * Consistency property level for Cassandra write operations.
    +   */
    +  private static final String WRITE_OP_CL = "write.consistency.level";
    +
    +  public static final Logger LOG = LoggerFactory.getLogger(CassandraStore.class);
    +
    +  private Cluster cluster;
    +
    +  private Session session;
    +
    +  public CassandraStore() {
    +    super();
    +  }
    +
    +  @Override
    +  public void initialize(Class<K> keyClass, Class<T> persistentClass, Properties
properties) {
    +    LOG.debug("Initializing Cassandra store");
    +    super.initialize(keyClass, persistentClass, properties);
    +    try {
    +      Cluster.Builder builder = Cluster.builder();
    +      populateSettings(builder, properties);
    +      this.cluster = builder.build();
    +      this.session = this.cluster.connect();
    +    } catch (Exception e) {
    +      LOG.error("Error while initializing Cassandra store: {}",
    +              new Object[]{e.getMessage()});
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private void populateSettings(Cluster.Builder builder, Properties properties) {
    +    String serversParam = properties.getProperty(Constants.CASSANDRA_SERVERS);
    +    String[] servers = serversParam.split(",");
    +    for (String server : servers) {
    +      builder = builder.addContactPoint(server);
    +    }
    +    String portProp = properties.getProperty(Constants.PORT);
    +    if (portProp != null) {
    +      builder = builder.withPort(Integer.parseInt(portProp));
    +    }
    +    String clusterNameProp = properties.getProperty(Constants.CLUSTER_NAME);
    +    if (clusterNameProp != null) {
    +      builder = builder.withClusterName(clusterNameProp);
    +    }
    +    String compressionProp = properties.getProperty(Constants.COMPRESSION);
    +    if (compressionProp != null) {
    +      builder = builder.withCompression(ProtocolOptions.Compression.valueOf(compressionProp));
    +    }
    +    builder = this.populateCredentials(properties, builder);
    +    builder = this.populateLoadBalancingProp(properties, builder);
    +    String enableJMXProp = properties.getProperty(Constants.ENABLE_JMX_REPORTING);
    +    if (enableJMXProp != null) {
    +      if (!Boolean.parseBoolean(enableJMXProp)) {
    +        builder = builder.withoutJMXReporting();
    +      }
    +    }
    +    String enableMetricsProp = properties.getProperty(Constants.ENABLE_METRICS);
    +    if (enableMetricsProp != null) {
    +      if (!Boolean.parseBoolean(enableMetricsProp)) {
    +        builder = builder.withoutMetrics();
    +      }
    +    }
    +    builder = this.populatePoolingSettings(properties, builder);
    +    String versionProp = properties.getProperty(Constants.PROTOCOL_VERSION);
    +    if (versionProp != null) {
    +      builder = builder.withProtocolVersion(ProtocolVersion.fromInt(Integer.parseInt(versionProp)));
    +    }
    +    builder = this.populateQueryOptions(properties, builder);
    +    builder = this.populateReconnectPolicy(properties, builder);
    +    builder = this.populateRetrytPolicy(properties, builder);
    +    builder = this.populateSocketOptions(properties, builder);
    +    String enableSSLProp = properties.getProperty(Constants.ENABLE_SSL);
    +    if (enableSSLProp != null) {
    +      if (Boolean.parseBoolean(enableSSLProp)) {
    +        builder = builder.withSSL();
    +      }
    +    }
    +  }
    +
    +
    +  private Cluster.Builder populateLoadBalancingProp(Properties properties, Cluster.Builder
builder) {
    +    String loadBalancingProp = properties.getProperty(Constants.LOAD_BALANCING_POLICY);
    +    if (loadBalancingProp != null) {
    +      switch (loadBalancingProp) {
    +        case "LatencyAwareRoundRobinPolicy":
    +          builder = builder.withLoadBalancingPolicy(LatencyAwarePolicy.builder(new RoundRobinPolicy()).build());
    +          break;
    +        case "RoundRobinPolicy":
    +          builder = builder.withLoadBalancingPolicy(new RoundRobinPolicy());
    +          break;
    +        case "DCAwareRoundRobinPolicy": {
    +          String dataCenter = properties.getProperty(Constants.DATA_CENTER);
    +          boolean allowRemoteDCsForLocalConsistencyLevel = Boolean.parseBoolean(
    +                  properties.getProperty(Constants.ALLOW_REMOTE_DCS_FOR_LOCAL_CONSISTENCY_LEVEL));
    +          if (dataCenter != null && !dataCenter.isEmpty()) {
    +            if (allowRemoteDCsForLocalConsistencyLevel) {
    +              builder = builder.withLoadBalancingPolicy(
    +                      DCAwareRoundRobinPolicy.builder().withLocalDc(dataCenter)
    +                              .allowRemoteDCsForLocalConsistencyLevel().build());
    +            } else {
    +              builder = builder.withLoadBalancingPolicy(
    +                      DCAwareRoundRobinPolicy.builder().withLocalDc(dataCenter).build());
    +            }
    +          } else {
    +            if (allowRemoteDCsForLocalConsistencyLevel) {
    +              builder = builder.withLoadBalancingPolicy(
    +                      (DCAwareRoundRobinPolicy.builder().allowRemoteDCsForLocalConsistencyLevel().build()));
    +            } else {
    +              builder = builder.withLoadBalancingPolicy((DCAwareRoundRobinPolicy.builder().build()));
    +            }
    +          }
    +          break;
    +        }
    +        case "TokenAwareRoundRobinPolicy":
    +          builder = builder.withLoadBalancingPolicy(new TokenAwarePolicy(new RoundRobinPolicy()));
    +          break;
    +        case "TokenAwareDCAwareRoundRobinPolicy": {
    +          String dataCenter = properties.getProperty(Constants.DATA_CENTER);
    +          boolean allowRemoteDCsForLocalConsistencyLevel = Boolean.parseBoolean(
    +                  properties.getProperty(Constants.ALLOW_REMOTE_DCS_FOR_LOCAL_CONSISTENCY_LEVEL));
    +          if (dataCenter != null && !dataCenter.isEmpty()) {
    +            if (allowRemoteDCsForLocalConsistencyLevel) {
    +              builder = builder.withLoadBalancingPolicy(new TokenAwarePolicy(
    +                      DCAwareRoundRobinPolicy.builder().withLocalDc(dataCenter)
    +                              .allowRemoteDCsForLocalConsistencyLevel().build()));
    +            } else {
    +              builder = builder.withLoadBalancingPolicy(new TokenAwarePolicy(
    +                      DCAwareRoundRobinPolicy.builder().withLocalDc(dataCenter).build()));
    +            }
    +          } else {
    +            if (allowRemoteDCsForLocalConsistencyLevel) {
    +              builder = builder.withLoadBalancingPolicy(new TokenAwarePolicy(
    +                      DCAwareRoundRobinPolicy.builder().allowRemoteDCsForLocalConsistencyLevel().build()));
    +            } else {
    +              builder = builder.withLoadBalancingPolicy(
    +                      new TokenAwarePolicy(DCAwareRoundRobinPolicy.builder().build()));
    +            }
    +          }
    +          break;
    +        }
    +        default:
    +          LOG.error("Unsupported Cassandra load balancing " + "policy: " + loadBalancingProp);
    +          break;
    +      }
    +    }
    +    return builder;
    +  }
    +
    +  private Cluster.Builder populateCredentials(Properties properties, Cluster.Builder
builder) {
    +    String usernameProp = properties.getProperty(Constants.USERNAME);
    +    String passwordProp = properties.getProperty(Constants.PASSWORD);
    +    if (usernameProp != null) {
    +      builder = builder.withCredentials(usernameProp, passwordProp);
    +    }
    +    return builder;
    +  }
    +
    +  private Cluster.Builder populatePoolingSettings(Properties properties, Cluster.Builder
builder) {
    +    String localCoreConnectionsPerHost = properties.getProperty(Constants.LOCAL_CORE_CONNECTIONS_PER_HOST);
    +    String remoteCoreConnectionsPerHost = properties.getProperty(Constants.REMOTE_CORE_CONNECTIONS_PER_HOST);
    +    String localMaxConnectionsPerHost = properties.getProperty(Constants.LOCAL_MAX_CONNECTIONS_PER_HOST);
    +    String remoteMaxConnectionsPerHost = properties.getProperty(Constants.REMOTE_MAX_CONNECTIONS_PER_HOST);
    +    String localNewConnectionThreshold = properties.getProperty(Constants.LOCAL_NEW_CONNECTION_THRESHOLD);
    +    String remoteNewConnectionThreshold = properties.getProperty(Constants.REMOTE_NEW_CONNECTION_THRESHOLD);
    +    String localMaxRequestsPerConnection = properties.getProperty(Constants.LOCAL_MAX_REQUESTS_PER_CONNECTION);
    +    String remoteMaxRequestsPerConnection = properties.getProperty(Constants.REMOTE_MAX_REQUESTS_PER_CONNECTION);
    +    PoolingOptions options = new PoolingOptions();
    +    if (localCoreConnectionsPerHost != null) {
    +      options.setCoreConnectionsPerHost(HostDistance.LOCAL, Integer.parseInt(localCoreConnectionsPerHost));
    +    }
    +    if (remoteCoreConnectionsPerHost != null) {
    +      options.setCoreConnectionsPerHost(HostDistance.REMOTE, Integer.parseInt(remoteCoreConnectionsPerHost));
    +    }
    +    if (localMaxConnectionsPerHost != null) {
    +      options.setMaxConnectionsPerHost(HostDistance.LOCAL, Integer.parseInt(localMaxConnectionsPerHost));
    +    }
    +    if (remoteMaxConnectionsPerHost != null) {
    +      options.setMaxConnectionsPerHost(HostDistance.REMOTE, Integer.parseInt(remoteMaxConnectionsPerHost));
    +    }
    +    if (localNewConnectionThreshold != null) {
    +      options.setNewConnectionThreshold(HostDistance.LOCAL, Integer.parseInt(localNewConnectionThreshold));
    +    }
    +    if (remoteNewConnectionThreshold != null) {
    +      options.setNewConnectionThreshold(HostDistance.REMOTE, Integer.parseInt(remoteNewConnectionThreshold));
    +    }
    +    if (localMaxRequestsPerConnection != null) {
    +      options.setMaxRequestsPerConnection(HostDistance.LOCAL, Integer.parseInt(localMaxRequestsPerConnection));
    +    }
    +    if (remoteMaxRequestsPerConnection != null) {
    +      options.setMaxRequestsPerConnection(HostDistance.REMOTE, Integer.parseInt(remoteMaxRequestsPerConnection));
    +    }
    +    builder = builder.withPoolingOptions(options);
    +    return builder;
    +  }
    +
    +  private Cluster.Builder populateQueryOptions(Properties properties, Cluster.Builder
builder) {
    +    String consistencyLevelProp = properties.getProperty(Constants.CONSISTENCY_LEVEL);
    +    String serialConsistencyLevelProp = properties.getProperty(Constants.SERIAL_CONSISTENCY_LEVEL);
    +    String fetchSize = properties.getProperty(Constants.FETCH_SIZE);
    +    QueryOptions options = new QueryOptions();
    +    if (consistencyLevelProp != null) {
    +      options.setConsistencyLevel(ConsistencyLevel.valueOf(consistencyLevelProp));
    +    }
    +    if (serialConsistencyLevelProp != null) {
    +      options.setSerialConsistencyLevel(ConsistencyLevel.valueOf(serialConsistencyLevelProp));
    +    }
    +    if (fetchSize != null) {
    +      options.setFetchSize(Integer.parseInt(fetchSize));
    +    }
    +    return builder.withQueryOptions(options);
    +  }
    +
    +  private Cluster.Builder populateReconnectPolicy(Properties properties, Cluster.Builder
builder) {
    +    String reconnectionPolicy = properties.getProperty(Constants.RECONNECTION_POLICY);
    +    if (reconnectionPolicy != null) {
    +      switch (reconnectionPolicy) {
    +        case "ConstantReconnectionPolicy": {
    +          String constantReconnectionPolicyDelay = properties.getProperty(Constants.CONSTANT_RECONNECTION_POLICY_DELAY);
    +
    +          ConstantReconnectionPolicy policy = new ConstantReconnectionPolicy(Long.parseLong(constantReconnectionPolicyDelay));
    +          builder = builder.withReconnectionPolicy(policy);
    +          break;
    +        }
    +        case "ExponentialReconnectionPolicy": {
    +          String exponentialReconnectionPolicyBaseDelay = properties.getProperty(Constants.EXPONENTIAL_RECONNECTION_POLICY_BASE_DELAY);
    +          String exponentialReconnectionPolicyMaxDelay = properties.getProperty(Constants.EXPONENTIAL_RECONNECTION_POLICY_MAX_DELAY);
    +
    +          ExponentialReconnectionPolicy policy = new ExponentialReconnectionPolicy(Long.parseLong(exponentialReconnectionPolicyBaseDelay),
    +                  Long.parseLong(exponentialReconnectionPolicyMaxDelay));
    +          builder = builder.withReconnectionPolicy(policy);
    +          break;
    +        }
    +        default:
    +          LOG.error("Unsupported reconnection policy " + reconnectionPolicy);
    +      }
    +    }
    +    return builder;
    +  }
    +
    +  private Cluster.Builder populateRetrytPolicy(Properties properties, Cluster.Builder
builder) {
    +    String retryPolicy = properties.getProperty(Constants.RETRY_POLICY);
    +    if (retryPolicy != null) {
    +      switch (retryPolicy) {
    +        case "DefaultRetryPolicy":
    +          builder = builder.withRetryPolicy(DefaultRetryPolicy.INSTANCE);
    +          break;
    +        case "DowngradingConsistencyRetryPolicy":
    +          builder = builder.withRetryPolicy(DowngradingConsistencyRetryPolicy.INSTANCE);
    +          break;
    +        case "FallthroughRetryPolicy":
    +          builder = builder.withRetryPolicy(FallthroughRetryPolicy.INSTANCE);
    +          break;
    +        case "LoggingDefaultRetryPolicy":
    +          builder = builder.withRetryPolicy(new LoggingRetryPolicy(DefaultRetryPolicy.INSTANCE));
    +          break;
    +        case "LoggingDowngradingConsistencyRetryPolicy":
    +          builder = builder.withRetryPolicy(new LoggingRetryPolicy(DowngradingConsistencyRetryPolicy.INSTANCE));
    +          break;
    +        case "LoggingFallthroughRetryPolicy":
    +          builder = builder.withRetryPolicy(new LoggingRetryPolicy(FallthroughRetryPolicy.INSTANCE));
    +          break;
    +        default:
    +          LOG.error("Unsupported retry policy " + retryPolicy);
    +          break;
    +      }
    +    }
    +    return builder;
    +  }
    +
    +  private Cluster.Builder populateSocketOptions(Properties properties, Cluster.Builder
builder) {
    +    String connectionTimeoutMillisProp = properties.getProperty(Constants.CONNECTION_TIMEOUT_MILLIS);
    +    String keepAliveProp = properties.getProperty(Constants.KEEP_ALIVE);
    +    String readTimeoutMillisProp = properties.getProperty(Constants.READ_TIMEOUT_MILLIS);
    +    String receiveBufferSizeProp = properties.getProperty(Constants.RECEIVER_BUFFER_SIZE);
    +    String reuseAddress = properties.getProperty(Constants.REUSE_ADDRESS);
    +    String sendBufferSize = properties.getProperty(Constants.SEND_BUFFER_SIZE);
    +    String soLinger = properties.getProperty(Constants.SO_LINGER);
    +    String tcpNoDelay = properties.getProperty(Constants.TCP_NODELAY);
    +    SocketOptions options = new SocketOptions();
    +    if (connectionTimeoutMillisProp != null) {
    +      options.setConnectTimeoutMillis(Integer.parseInt(connectionTimeoutMillisProp));
    +    }
    +    if (keepAliveProp != null) {
    +      options.setKeepAlive(Boolean.parseBoolean(keepAliveProp));
    +    }
    +    if (readTimeoutMillisProp != null) {
    +      options.setReadTimeoutMillis(Integer.parseInt(readTimeoutMillisProp));
    +    }
    +    if (receiveBufferSizeProp != null) {
    +      options.setReceiveBufferSize(Integer.parseInt(receiveBufferSizeProp));
    +    }
    +    if (reuseAddress != null) {
    +      options.setReuseAddress(Boolean.parseBoolean(reuseAddress));
    +    }
    +    if (sendBufferSize != null) {
    +      options.setSendBufferSize(Integer.parseInt(sendBufferSize));
    +    }
    +    if (soLinger != null) {
    +      options.setSoLinger(Integer.parseInt(soLinger));
    +    }
    +    if (tcpNoDelay != null) {
    +      options.setTcpNoDelay(Boolean.parseBoolean(tcpNoDelay));
    +    }
    +    return builder.withSocketOptions(options);
    +  }
    +
    +  @Override
    +  public void setPersistentClass(Class<T> persistentClass) {
    --- End diff --
    
    Please make sure that all public methods are documented.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Mime
View raw message