private static void cleanup() throws IOException { // clean up commitlog String[] directoryNames = { DatabaseDescriptor.getCommitLogLocation(), }; for (String dirName : directoryNames) { File dir = new File(dirName); if (!dir.exists()) { log.error("No such directory: " + dir.getAbsolutePath()); throw new RuntimeException("No such directory: " + dir.getAbsolutePath()); } FileUtils.deleteRecursive(dir); } // clean up data directory which are stored as data directory/table/data // files for (String dirName : DatabaseDescriptor.getAllDataFileLocations()) { File dir = new File(dirName); if (!dir.exists()) { log.error("No such directory: " + dir.getAbsolutePath()); throw new RuntimeException("No such directory: " + dir.getAbsolutePath()); } FileUtils.deleteRecursive(dir); } }
public void get() throws WriteTimeoutException, WriteFailureException { long requestTimeout = writeType == WriteType.COUNTER ? DatabaseDescriptor.getCounterWriteRpcTimeout() : DatabaseDescriptor.getWriteRpcTimeout(); long timeout = TimeUnit.MILLISECONDS.toNanos(requestTimeout) - (System.nanoTime() - start); boolean success; try { success = condition.await(timeout, TimeUnit.NANOSECONDS); } catch (InterruptedException ex) { throw new AssertionError(ex); } if (!success) { int blockedFor = totalBlockFor(); int acks = ackCount(); // It's pretty unlikely, but we can race between exiting await above and here, so // that we could now have enough acks. In that case, we "lie" on the acks count to // avoid sending confusing info to the user (see CASSANDRA-6491). if (acks >= blockedFor) acks = blockedFor - 1; throw new WriteTimeoutException(writeType, consistencyLevel, acks, blockedFor); } if (totalBlockFor() + failures > totalEndpoints()) { throw new WriteFailureException( consistencyLevel, ackCount(), failures, totalBlockFor(), writeType); } }
protected int determineBlockFor(String table) { int blockFor = 0; switch (consistencyLevel) { case ONE: blockFor = 1; break; case ANY: blockFor = 1; break; case QUORUM: blockFor = (writeEndpoints.size() / 2) + 1; break; case ALL: blockFor = writeEndpoints.size(); break; default: throw new UnsupportedOperationException( "invalid consistency level: " + consistencyLevel.toString()); } // at most one node per range can bootstrap at a time, and these will be added to the write // until // bootstrap finishes (at which point we no longer need to write to the old ones). assert 1 <= blockFor && blockFor <= 2 * DatabaseDescriptor.getReplicationFactor(table) : String.format( "invalid response count %d for replication factor %d", blockFor, DatabaseDescriptor.getReplicationFactor(table)); return blockFor; }
/** Start the gossiper with the generation # retrieved from the System table */ public void start(InetAddress localEndpoint, int generationNbr) { localEndpoint_ = localEndpoint; /* Get the seeds from the config and initialize them. */ Set<InetAddress> seedHosts = DatabaseDescriptor.getSeeds(); for (InetAddress seed : seedHosts) { if (seed.equals(localEndpoint)) continue; seeds_.add(seed); } /* initialize the heartbeat state for this localEndpoint */ EndpointState localState = endpointStateMap_.get(localEndpoint_); if (localState == null) { HeartBeatState hbState = new HeartBeatState(generationNbr); localState = new EndpointState(hbState); localState.isAlive(true); localState.isAGossiper(true); endpointStateMap_.put(localEndpoint_, localState); } // notify snitches that Gossiper is about to start DatabaseDescriptor.getEndpointSnitch().gossiperStarting(); scheduledGossipTask = StorageService.scheduledTasks.scheduleWithFixedDelay( new GossipTask(), Gossiper.intervalInMillis_, Gossiper.intervalInMillis_, TimeUnit.MILLISECONDS); }
public String get_string_property(String propertyName) { if (propertyName.equals("cluster name")) { return DatabaseDescriptor.getClusterName(); } else if (propertyName.equals("config file")) { String filename = DatabaseDescriptor.getConfigFileName(); try { StringBuilder fileData = new StringBuilder(8192); BufferedInputStream stream = new BufferedInputStream(new FileInputStream(filename)); byte[] buf = new byte[1024]; int numRead; while ((numRead = stream.read(buf)) != -1) { String str = new String(buf, 0, numRead); fileData.append(str); } stream.close(); return fileData.toString(); } catch (IOException e) { return "file not found!"; } } else if (propertyName.equals("version")) { return "0.3.0"; } else { return "?"; } }
protected MessagingService() { for (ReservedVerbs_ verbs : ReservedVerbs_.values()) { reservedVerbs_.put(verbs.toString(), verbs.toString()); } verbHandlers_ = new HashMap<String, IVerbHandler>(); endPoints_ = new HashSet<EndPoint>(); /* * Leave callbacks in the cachetable long enough that any related messages will arrive * before the callback is evicted from the table. The concurrency level is set at 128 * which is the sum of the threads in the pool that adds shit into the table and the * pool that retrives the callback from here. */ int maxSize = MessagingConfig.getMessagingThreadCount(); callbackMap_ = new Cachetable<String, IAsyncCallback>(2 * DatabaseDescriptor.getRpcTimeout()); taskCompletionMap_ = new Cachetable<String, IAsyncResult>(2 * DatabaseDescriptor.getRpcTimeout()); messageDeserializationExecutor_ = new DebuggableThreadPoolExecutor( maxSize, maxSize, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new ThreadFactoryImpl("MESSAGING-SERVICE-POOL")); messageSerializerExecutor_ = new DebuggableThreadPoolExecutor( maxSize, maxSize, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new ThreadFactoryImpl("MESSAGE-SERIALIZER-POOL")); messageDeserializerExecutor_ = new DebuggableThreadPoolExecutor( maxSize, maxSize, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new ThreadFactoryImpl("MESSAGE-DESERIALIZER-POOL")); streamExecutor_ = new DebuggableThreadPoolExecutor( 1, 1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new ThreadFactoryImpl("MESSAGE-STREAMING-POOL")); protocol_ = hash(HashingSchemes.MD5, "FB-MESSAGING".getBytes()); /* register the response verb handler */ registerVerbHandlers(MessagingService.responseVerbHandler_, new ResponseVerbHandler()); /* register stage for response */ StageManager.registerStage( MessagingService.responseStage_, new MultiThreadedStage("RESPONSE-STAGE", maxSize)); }
public DropKeyspace(String name, boolean blockOnFileDeletion) throws ConfigurationException, IOException { super( UUIDGen.makeType1UUIDFromHost(FBUtilities.getLocalAddress()), DatabaseDescriptor.getDefsVersion()); this.name = name; this.blockOnFileDeletion = blockOnFileDeletion; KSMetaData ksm = DatabaseDescriptor.getTableDefinition(name); if (ksm == null) throw new ConfigurationException("Keyspace does not exist."); rm = makeDefinitionMutation(null, ksm, newVersion); }
/** * Perform recovery on commit logs located in the directory specified by the config file. * * @return the number of mutations replayed */ public int recover() throws IOException { // If createReserveSegments is already flipped, the CLSM is running and recovery has already // taken place. if (allocator.createReserveSegments) return 0; // Allocator could be in the process of initial startup with 0 active and available segments. We // need to wait for // the allocation manager to finish allocation and add it to available segments so we don't get // an invalid response // on allocator.manages(...) below by grabbing a file off the filesystem before it's added to // the CLQ. allocator.allocatingFrom(); FilenameFilter unmanagedFilesFilter = new FilenameFilter() { public boolean accept(File dir, String name) { // we used to try to avoid instantiating commitlog (thus creating an empty segment ready // for writes) // until after recover was finished. this turns out to be fragile; it is less // error-prone to go // ahead and allow writes before recover(), and just skip active segments when we do. return CommitLogDescriptor.isValid(name) && !allocator.manages(name); } }; // submit all existing files in the commit log dir for archiving prior to recovery - // CASSANDRA-6904 for (File file : new File(DatabaseDescriptor.getCommitLogLocation()).listFiles(unmanagedFilesFilter)) { archiver.maybeArchive(file.getPath(), file.getName()); archiver.maybeWaitForArchiving(file.getName()); } assert archiver.archivePending.isEmpty() : "Not all commit log archive tasks were completed before restore"; archiver.maybeRestoreArchive(); File[] files = new File(DatabaseDescriptor.getCommitLogLocation()).listFiles(unmanagedFilesFilter); int replayed = 0; if (files.length == 0) { logger.info("No commitlog files found; skipping replay"); } else { Arrays.sort(files, new CommitLogSegmentFileComparator()); logger.info("Replaying {}", StringUtils.join(files, ", ")); replayed = recover(files); logger.info("Log replay complete, {} replayed mutations", replayed); for (File f : files) allocator.recycleSegment(f); } allocator.enableReserveSegmentCreation(); return replayed; }
public IncomingTcpConnection(int version, boolean compressed, Socket socket) { assert socket != null; this.version = version; this.compressed = compressed; this.socket = socket; if (DatabaseDescriptor.getInternodeRecvBufferSize() != null) { try { this.socket.setReceiveBufferSize(DatabaseDescriptor.getInternodeRecvBufferSize()); } catch (SocketException se) { logger.warn("Failed to set receive buffer size on internode socket.", se); } } }
public IncomingTcpConnection( int version, boolean compressed, Socket socket, Set<Closeable> group) { super("MessagingService-Incoming-" + socket.getInetAddress()); this.version = version; this.compressed = compressed; this.socket = socket; this.group = group; if (DatabaseDescriptor.getInternodeRecvBufferSize() != null) { try { this.socket.setReceiveBufferSize(DatabaseDescriptor.getInternodeRecvBufferSize()); } catch (SocketException se) { logger.warn("Failed to set receive buffer size on internode socket.", se); } } }
public static SSTableFormat.Type getFormatType(Collection<SSTableReader> sstables) { if (sstables.isEmpty() || !SSTableFormat.enableSSTableDevelopmentTestMode) return DatabaseDescriptor.getSSTableFormat(); // Allows us to test compaction of non-default formats return sstables.iterator().next().descriptor.formatType; }
private void setup() throws IOException, TTransportException { int listenPort = DatabaseDescriptor.getThriftPort(); Thread.setDefaultUncaughtExceptionHandler( new Thread.UncaughtExceptionHandler() { public void uncaughtException(Thread t, Throwable e) { logger.error("Fatal exception in thread " + t, e); } }); CassandraServer peerStorageServer = new CassandraServer(); peerStorageServer.start(); Cassandra.Processor processor = new Cassandra.Processor(peerStorageServer); // Transport TServerSocket tServerSocket = new TServerSocket(new InetSocketAddress(FBUtilities.getHostName(), listenPort)); // Protocol factory TProtocolFactory tProtocolFactory = new TBinaryProtocol.Factory(); // ThreadPool Server TThreadPoolServer.Options options = new TThreadPoolServer.Options(); options.minWorkerThreads = 64; serverEngine = new TThreadPoolServer( new TProcessorFactory(processor), tServerSocket, new TTransportFactory(), new TTransportFactory(), tProtocolFactory, tProtocolFactory, options); }
public PaxosResponseType get() { long timeout = DatabaseDescriptor.getRpcTimeout() - (System.currentTimeMillis() - startTime); boolean success; try { success = condition.await(timeout, TimeUnit.MILLISECONDS); } catch (InterruptedException ex) { throw new AssertionError(ex); } if (!success) { return PaxosResponseType.Timeout; } if (nackcount.get() >= expectedResponses) return PaxosResponseType.Nack; // normal paxos phase 2 if (expectedValue != null) { // System.out.println("AcceptResponseHandler : Quorum"); return PaxosResponseType.Quorum; } // no-op phase 2 else { for (Entry<IPaxosValue, AtomicInteger> entry : responseValues.entrySet()) { if (entry.getValue().get() >= expectedResponses) { this.acceptedValue = entry.getKey(); return PaxosResponseType.Quorum; } } this.acceptedValue = null; return PaxosResponseType.NoConsensus; } }
public void validate(String name) { // Attempt to instantiate the ARS, which will throw a ConfigurationException if the options // aren't valid. TokenMetadata tmd = StorageService.instance.getTokenMetadata(); IEndpointSnitch eps = DatabaseDescriptor.getEndpointSnitch(); AbstractReplicationStrategy.validateReplicationStrategy(name, klass, tmd, eps, options); }
@Test public void testCompactions() throws IOException, ExecutionException, InterruptedException { CompactionManager.instance.disableAutoCompaction(); // this test does enough rows to force multiple block indexes to be used Table table = Table.open(TABLE1); ColumnFamilyStore store = table.getColumnFamilyStore("Standard1"); final int ROWS_PER_SSTABLE = 10; Set<DecoratedKey> inserted = new HashSet<DecoratedKey>(); for (int j = 0; j < (DatabaseDescriptor.getIndexInterval() * 3) / ROWS_PER_SSTABLE; j++) { for (int i = 0; i < ROWS_PER_SSTABLE; i++) { DecoratedKey key = Util.dk(String.valueOf(i % 2)); RowMutation rm = new RowMutation(TABLE1, key.key); rm.add( new QueryPath("Standard1", null, ByteBufferUtil.bytes(String.valueOf(i / 2))), ByteBufferUtil.EMPTY_BYTE_BUFFER, j * ROWS_PER_SSTABLE + i); rm.apply(); inserted.add(key); } store.forceBlockingFlush(); assertEquals(inserted.toString(), inserted.size(), Util.getRangeSlice(store).size()); } while (true) { Future<Integer> ft = CompactionManager.instance.submitMinorIfNeeded(store); if (ft.get() == 0) break; } if (store.getSSTables().size() > 1) { CompactionManager.instance.performMajor(store); } assertEquals(inserted.size(), Util.getRangeSlice(store).size()); }
public DropColumnFamily(String tableName, String cfName) throws ConfigurationException, IOException { super( UUIDGen.makeType1UUIDFromHost(FBUtilities.getLocalAddress()), DatabaseDescriptor.getDefsVersion()); this.tableName = tableName; this.cfName = cfName; KSMetaData ksm = DatabaseDescriptor.getTableDefinition(tableName); if (ksm == null) throw new ConfigurationException("Keyspace does not already exist."); else if (!ksm.cfMetaData().containsKey(cfName)) throw new ConfigurationException("CF is not defined in that keyspace."); KSMetaData newKsm = makeNewKeyspaceDefinition(ksm); rm = Migration.makeDefinitionMutation(newKsm, null, newVersion); }
private static void validateColumns( String keyspace, String columnFamilyName, ByteBuffer superColumnName, Iterable<ByteBuffer> column_names) throws InvalidRequestException { if (superColumnName != null) { if (superColumnName.remaining() > IColumn.MAX_NAME_LENGTH) throw new InvalidRequestException( "supercolumn name length must not be greater than " + IColumn.MAX_NAME_LENGTH); if (superColumnName.remaining() == 0) throw new InvalidRequestException("supercolumn name must not be empty"); if (DatabaseDescriptor.getColumnFamilyType(keyspace, columnFamilyName) == ColumnFamilyType.Standard) throw new InvalidRequestException( "supercolumn specified to ColumnFamily " + columnFamilyName + " containing normal columns"); } AbstractType comparator = ColumnFamily.getComparatorFor(keyspace, columnFamilyName, superColumnName); for (ByteBuffer name : column_names) { if (name.remaining() > IColumn.MAX_NAME_LENGTH) throw new InvalidRequestException( "column name length must not be greater than " + IColumn.MAX_NAME_LENGTH); if (name.remaining() == 0) throw new InvalidRequestException("column name must not be empty"); try { comparator.validate(name); } catch (MarshalException e) { throw new InvalidRequestException(e.getMessage()); } } }
public Map<String, Map<String, String>> describe_keyspace(String table) throws NotFoundException { Map<String, Map<String, String>> columnFamiliesMap = new HashMap<String, Map<String, String>>(); Map<String, CFMetaData> tableMetaData = DatabaseDescriptor.getTableMetaData(table); // table doesn't exist if (tableMetaData == null) { throw new NotFoundException(); } Iterator iter = tableMetaData.entrySet().iterator(); while (iter.hasNext()) { Map.Entry<String, CFMetaData> pairs = (Map.Entry<String, CFMetaData>) iter.next(); CFMetaData columnFamilyMetaData = pairs.getValue(); String desc = ""; Map<String, String> columnMap = new HashMap<String, String>(); desc = columnFamilyMetaData.n_columnMap + "(" + columnFamilyMetaData.n_columnKey + ", " + columnFamilyMetaData.n_columnValue + ", " + columnFamilyMetaData.n_columnTimestamp + ")"; if (columnFamilyMetaData.columnType.equals("Super")) { columnMap.put("Type", "Super"); desc = columnFamilyMetaData.n_superColumnMap + "(" + columnFamilyMetaData.n_superColumnKey + ", " + desc + ")"; } else { columnMap.put("Type", "Standard"); } desc = columnFamilyMetaData.tableName + "." + columnFamilyMetaData.cfName + "(" + columnFamilyMetaData.n_rowKey + ", " + desc + ")"; columnMap.put("Desc", desc); columnMap.put("CompareWith", columnFamilyMetaData.comparator.getClass().getName()); if (columnFamilyMetaData.columnType.equals("Super")) { columnMap.put( "CompareSubcolumnsWith", columnFamilyMetaData.subcolumnComparator.getClass().getName()); } columnMap.put("FlushPeriodInMinutes", columnFamilyMetaData.flushPeriodInMinutes + ""); columnFamiliesMap.put(columnFamilyMetaData.cfName, columnMap); } return columnFamiliesMap; }
public static void recover() throws IOException { String directory = DatabaseDescriptor.getCommitLogLocation(); File[] files = new File(directory) .listFiles( new FilenameFilter() { public boolean accept(File dir, String name) { return CommitLogSegment.possibleCommitLogFile(name); } }); if (files.length == 0) return; Arrays.sort(files, new FileUtils.FileComparator()); logger.info("Replaying " + StringUtils.join(files, ", ")); recover(files); for (File f : files) { FileUtils.delete( CommitLogHeader.getHeaderPathFromSegmentPath( f.getAbsolutePath())); // may not actually exist if (!f.delete()) logger.error( "Unable to remove " + f + "; you should remove it manually or next restart will replay it again (harmless, but time-consuming)"); } logger.info("Log replay complete"); }
public List<SuperColumn> get_slice_super( String table, String key, String column_family, byte[] start, byte[] finish, boolean is_ascending, int count, int consistency_level) throws InvalidRequestException { if (logger.isDebugEnabled()) logger.debug("get_slice_super"); if (!DatabaseDescriptor.getColumnFamilyType(table, column_family).equals("Super")) throw new InvalidRequestException("get_slice_super requires a super CF name"); if (count <= 0) throw new InvalidRequestException("get_slice_super requires positive count"); ColumnFamily cfamily = readColumnFamily( new SliceFromReadCommand( table, key, new QueryPath(column_family), start, finish, is_ascending, count), consistency_level); if (cfamily == null) { return EMPTY_SUPERCOLUMNS; } Collection<IColumn> columns = cfamily.getSortedColumns(); return thriftifySuperColumns(columns, !is_ascending); }
public List<String> get_string_list_property(String propertyName) { if (propertyName.equals("keyspaces")) { return DatabaseDescriptor.getTables(); } else { return new ArrayList<String>(); } }
/** * The purpose of this test is to make sure that: When a TimedOutException is thrown out from * invoke method, the CassandraProxyClient will try at least maxAttempts times before throwing the * exception to the client. * * <p>A counter from the ThriftServer in BriskErrorDaemon tracks how many times the method has * been called. * * @throws Exception */ @Test public void testReconnect() throws Exception { EmbeddedBriskErrorServer.startBrisk(); Brisk.Iface client = CassandraProxyClient.newProxyConnection( "localhost", DatabaseDescriptor.getRpcPort(), true, ConnectionStrategy.STICKY); List<KsDef> ks = client.describe_keyspaces(); assertTrue(ks.size() > 0); try { client.get(ByteBufferUtil.EMPTY_BYTE_BUFFER, new ColumnPath("test"), ConsistencyLevel.ALL); fail("Expect a TimedoutException"); } catch (TimedOutException e) { // This is expected. } assertEquals( 11, client.get_count( ByteBufferUtil.EMPTY_BYTE_BUFFER, new ColumnParent("test"), new SlicePredicate(), ConsistencyLevel.ALL)); }
@Test public void testRoundTripStandardCf() throws IOException, ParseException { File tempSS = createTemporarySSTable("Keyspace1", "Standard1"); ColumnFamily cfamily = ColumnFamily.create("Keyspace1", "Standard1"); IPartitioner<?> partitioner = DatabaseDescriptor.getPartitioner(); DataOutputBuffer dob = new DataOutputBuffer(); SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, partitioner); // Add rowA cfamily.addColumn( new QueryPath("Standard1", null, "name".getBytes()), "val".getBytes(), 1, false); ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false); writer.append(partitioner.decorateKey("rowA"), dob); dob.reset(); cfamily.clear(); // Add rowExclude cfamily.addColumn( new QueryPath("Standard1", null, "name".getBytes()), "val".getBytes(), 1, false); ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false); writer.append(partitioner.decorateKey("rowExclude"), dob); dob.reset(); cfamily.clear(); SSTableReader reader = writer.closeAndOpenReader(); // Export to JSON and verify File tempJson = File.createTempFile("Standard1", ".json"); SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[] {"rowExclude"}); // Import JSON to another SSTable file File tempSS2 = createTemporarySSTable("Keyspace1", "Standard1"); SSTableImport.importJson(tempJson.getPath(), "Keyspace1", "Standard1", tempSS2.getPath()); reader = SSTableReader.open(tempSS2.getPath(), DatabaseDescriptor.getPartitioner()); NamesQueryFilter qf = new NamesQueryFilter("rowA", new QueryPath("Standard1", null, null), "name".getBytes()); ColumnFamily cf = qf.getSSTableColumnIterator(reader).getColumnFamily(); assertTrue(cf != null); assertTrue(Arrays.equals(cf.getColumn("name".getBytes()).value(), hexToBytes("76616c"))); qf = new NamesQueryFilter( "rowExclude", new QueryPath("Standard1", null, null), "name".getBytes()); cf = qf.getSSTableColumnIterator(reader).getColumnFamily(); assert cf == null; }
public void serialize(IndexSummary t, DataOutput out) throws IOException { out.writeInt(DatabaseDescriptor.getIndexInterval()); out.writeInt(t.keys.length); for (int i = 0; i < t.keys.length; i++) { out.writeLong(t.getPosition(i)); ByteBufferUtil.writeWithLength(t.keys[i], out); } }
/** * param @ table - name of table for which we are maintaining this commit log. param @ * recoverymode - is commit log being instantiated in in recovery mode. */ private CommitLog() { try { DatabaseDescriptor.createAllDirectories(); segmentSize = DatabaseDescriptor.getCommitLogSegmentSize(); } catch (IOException e) { throw new IOError(e); } // all old segments are recovered and deleted before CommitLog is instantiated. // All we need to do is create a new one. segments.add(new CommitLogSegment()); executor = DatabaseDescriptor.getCommitLogSync() == Config.CommitLogSync.batch ? new BatchCommitLogExecutorService() : new PeriodicCommitLogExecutorService(this); }
Message makeGossipDigestSynMessage(List<GossipDigest> gDigests) throws IOException { GossipDigestSynMessage gDigestMessage = new GossipDigestSynMessage(DatabaseDescriptor.getClusterName(), gDigests); ByteArrayOutputStream bos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(bos); GossipDigestSynMessage.serializer().serialize(gDigestMessage, dos); return new Message(localEndpoint_, StorageService.Verb.GOSSIP_DIGEST_SYN, bos.toByteArray()); }
@Test public void testExportSuperCf() throws IOException { File tempSS = createTemporarySSTable("Keyspace1", "Super4"); ColumnFamily cfamily = ColumnFamily.create("Keyspace1", "Super4"); IPartitioner<?> partitioner = DatabaseDescriptor.getPartitioner(); DataOutputBuffer dob = new DataOutputBuffer(); SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, partitioner); // Add rowA cfamily.addColumn( new QueryPath("Super4", "superA".getBytes(), "colA".getBytes()), "valA".getBytes(), 1, false); ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false); writer.append(partitioner.decorateKey("rowA"), dob); dob.reset(); cfamily.clear(); // Add rowB cfamily.addColumn( new QueryPath("Super4", "superB".getBytes(), "colB".getBytes()), "valB".getBytes(), 1, false); ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false); writer.append(partitioner.decorateKey("rowB"), dob); dob.reset(); cfamily.clear(); // Add rowExclude cfamily.addColumn( new QueryPath("Super4", "superX".getBytes(), "colX".getBytes()), "valX".getBytes(), 1, false); ColumnFamily.serializer().serializeWithIndexes(cfamily, dob, false); dob.reset(); cfamily.clear(); SSTableReader reader = writer.closeAndOpenReader(); // Export to JSON and verify File tempJson = File.createTempFile("Super4", ".json"); SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[] {"rowExclude"}); JSONObject json = (JSONObject) JSONValue.parse(new FileReader(tempJson)); JSONObject rowA = (JSONObject) json.get("rowA"); JSONObject superA = (JSONObject) rowA.get(cfamily.getComparator().getString("superA".getBytes())); JSONArray subColumns = (JSONArray) superA.get("subColumns"); JSONArray colA = (JSONArray) subColumns.get(0); JSONObject rowExclude = (JSONObject) json.get("rowExclude"); assert Arrays.equals(hexToBytes((String) colA.get(1)), "valA".getBytes()); assert !(Boolean) colA.get(3); assert rowExclude == null; }
protected Configuration buildConfiguration() { Configuration conf = new Configuration(); conf.set(CassandraClientHolder.CONF_PARAM_HOST, "localhost"); conf.setInt(CassandraClientHolder.CONF_PARAM_PORT, DatabaseDescriptor.getRpcPort()); conf.setBoolean(CassandraClientHolder.CONF_PARAM_FRAMED, true); conf.set(CassandraClientHolder.CONF_PARAM_CONNECTION_STRATEGY, "STICKY"); conf.set("hive.metastore.warehouse.dir", "cfs:///user/hive/warehouse"); return conf; }
public static void cleanup() { // clean up data directory which are stored as data directory/keyspace/data files for (String dirName : DatabaseDescriptor.getAllDataFileLocations()) { File dir = new File(dirName); if (!dir.exists()) continue; String[] children = dir.list(); for (String child : children) FileUtils.deleteRecursive(new File(dir, child)); } }
public ICache<RowCacheKey, IRowCacheEntry> create() { OHCacheBuilder<RowCacheKey, IRowCacheEntry> builder = OHCacheBuilder.newBuilder(); builder .capacity(DatabaseDescriptor.getRowCacheSizeInMB() * 1024 * 1024) .keySerializer(new KeySerializer()) .valueSerializer(new ValueSerializer()) .throwOOME(true); return new OHCacheAdapter(builder.build()); }