private static Cassandra.Client createThriftClient( String host, int port, String user, String passwd, ITransportFactory transportFactory) throws Exception { TTransport trans = transportFactory.openTransport(host, port); TProtocol protocol = new TBinaryProtocol(trans); Cassandra.Client client = new Cassandra.Client(protocol); if (user != null && passwd != null) { Map<String, String> credentials = new HashMap<>(); credentials.put(IAuthenticator.USERNAME_KEY, user); credentials.put(IAuthenticator.PASSWORD_KEY, passwd); AuthenticationRequest authenticationRequest = new AuthenticationRequest(credentials); client.login(authenticationRequest); } return client; }
private static void configureTransportFactory( ITransportFactory transportFactory, LoaderOptions opts) { Map<String, String> options = new HashMap<>(); // If the supplied factory supports the same set of options as our SSL impl, set those if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE)) options.put(SSLTransportFactory.TRUSTSTORE, opts.encOptions.truststore); if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE_PASSWORD)) options.put(SSLTransportFactory.TRUSTSTORE_PASSWORD, opts.encOptions.truststore_password); if (transportFactory.supportedOptions().contains(SSLTransportFactory.PROTOCOL)) options.put(SSLTransportFactory.PROTOCOL, opts.encOptions.protocol); if (transportFactory.supportedOptions().contains(SSLTransportFactory.CIPHER_SUITES)) options.put( SSLTransportFactory.CIPHER_SUITES, Joiner.on(',').join(opts.encOptions.cipher_suites)); if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE) && opts.encOptions.require_client_auth) options.put(SSLTransportFactory.KEYSTORE, opts.encOptions.keystore); if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE_PASSWORD) && opts.encOptions.require_client_auth) options.put(SSLTransportFactory.KEYSTORE_PASSWORD, opts.encOptions.keystore_password); // Now check if any of the factory's supported options are set as system properties for (String optionKey : transportFactory.supportedOptions()) if (System.getProperty(optionKey) != null) options.put(optionKey, System.getProperty(optionKey)); transportFactory.setOptions(options); }
public void handleStreamEvent(StreamEvent event) { if (event.eventType == StreamEvent.Type.STREAM_PREPARED) { SessionInfo session = ((StreamEvent.SessionPreparedEvent) event).session; sessionsByHost.put(session.peer, session); } else if (event.eventType == StreamEvent.Type.FILE_PROGRESS) { ProgressInfo progressInfo = ((StreamEvent.ProgressEvent) event).progress; // update progress Set<ProgressInfo> progresses = progressByHost.get(progressInfo.peer); if (progresses == null) { progresses = Sets.newSetFromMap(new ConcurrentHashMap<ProgressInfo, Boolean>()); progressByHost.put(progressInfo.peer, progresses); } if (progresses.contains(progressInfo)) progresses.remove(progressInfo); progresses.add(progressInfo); StringBuilder sb = new StringBuilder(); sb.append("\rprogress: "); long totalProgress = 0; long totalSize = 0; for (Map.Entry<InetAddress, Set<ProgressInfo>> entry : progressByHost.entrySet()) { SessionInfo session = sessionsByHost.get(entry.getKey()); long size = session.getTotalSizeToSend(); long current = 0; int completed = 0; for (ProgressInfo progress : entry.getValue()) { if (progress.currentBytes == progress.totalBytes) completed++; current += progress.currentBytes; } totalProgress += current; totalSize += size; sb.append("[").append(entry.getKey()); sb.append(" ").append(completed).append("/").append(session.getTotalFilesToSend()); sb.append(" (").append(size == 0 ? 100L : current * 100L / size).append("%)] "); } long time = System.nanoTime(); long deltaTime = Math.max(1L, TimeUnit.NANOSECONDS.toMillis(time - lastTime)); lastTime = time; long deltaProgress = totalProgress - lastProgress; lastProgress = totalProgress; sb.append("[total: ") .append(totalSize == 0 ? 100L : totalProgress * 100L / totalSize) .append("% - "); sb.append(mbPerSec(deltaProgress, deltaTime)).append("MB/s"); sb.append(" (avg: ") .append(mbPerSec(totalProgress, TimeUnit.NANOSECONDS.toMillis(time - start))) .append("MB/s)]"); System.out.print(sb.toString()); } }
@Override public void init(String keyspace) { Iterator<InetAddress> hostiter = hosts.iterator(); while (hostiter.hasNext()) { try { // Query endpoint to ranges map and schemas from thrift InetAddress host = hostiter.next(); Cassandra.Client client = createThriftClient( host.getHostAddress(), rpcPort, this.user, this.passwd, this.transportFactory); setPartitioner(client.describe_partitioner()); Token.TokenFactory tkFactory = getPartitioner().getTokenFactory(); for (TokenRange tr : client.describe_ring(keyspace)) { Range<Token> range = new Range<>( tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token), getPartitioner()); for (String ep : tr.endpoints) { addRangeForEndpoint(range, InetAddress.getByName(ep)); } } String query = String.format( "SELECT * FROM %s.%s WHERE keyspace_name = '%s'", Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, keyspace); CqlResult result = client.execute_cql3_query( ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE); for (CqlRow row : result.rows) { CFMetaData metadata = CFMetaData.fromThriftCqlRow(row); knownCfs.put(metadata.cfName, metadata); } break; } catch (Exception e) { if (!hostiter.hasNext()) throw new RuntimeException("Could not retrieve endpoint ranges: ", e); } } }
// we don't care about the return value but care about it throwing exception public void runMayThrow() throws Exception { logger.info( String.format( "[repair #%s] new session: will sync %s on range %s for %s.%s", getName(), repairedNodes(), range, tablename, Arrays.toString(cfnames))); if (endpoints.isEmpty()) { differencingDone.signalAll(); logger.info( String.format( "[repair #%s] No neighbors to repair with on range %s: session completed", getName(), range)); return; } // Checking all nodes are live for (InetAddress endpoint : endpoints) { if (!FailureDetector.instance.isAlive(endpoint)) { differencingDone.signalAll(); logger.info( String.format( "[repair #%s] Cannot proceed on repair because a neighbor (%s) is dead: session failed", getName(), endpoint)); return; } if (Gossiper.instance.getVersion(endpoint) < MessagingService.VERSION_11 && isSequential) { logger.info( String.format( "[repair #%s] Cannot repair using snapshots as node %s is pre-1.1", getName(), endpoint)); return; } } AntiEntropyService.instance.sessions.put(getName(), this); Gossiper.instance.register(this); FailureDetector.instance.registerFailureDetectionEventListener(this); try { // Create and queue a RepairJob for each column family for (String cfname : cfnames) { RepairJob job = new RepairJob(cfname); jobs.offer(job); activeJobs.put(cfname, job); } jobs.peek().sendTreeRequests(); // block whatever thread started this session until all requests have been returned: // if this thread dies, the session will still complete in the background completed.await(); if (exception == null) { logger.info(String.format("[repair #%s] session completed successfully", getName())); } else { logger.error( String.format("[repair #%s] session completed with the following error", getName()), exception); throw exception; } } catch (InterruptedException e) { throw new RuntimeException("Interrupted while waiting for repair."); } finally { // mark this session as terminated terminate(); FailureDetector.instance.unregisterFailureDetectionEventListener(this); Gossiper.instance.unregister(this); AntiEntropyService.instance.sessions.remove(getName()); } }