/** * 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.cassandra.streaming; import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.db.Table; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.AESRecoveryProcessor; import org.apache.cassandra.io.sstable.IRecoveryProcessor; import org.apache.cassandra.io.sstable.IndexRecoveryProcessor; import org.apache.cassandra.io.sstable.SSTableReader; import org.apache.cassandra.io.sstable.SSTableWriter; import org.apache.cassandra.net.MessagingService; /** * This is the callback handler that is invoked on the receiving node when a file changes status from RECEIVE to either * FileStatus.STREAM (needs to be restreamed) or FileStatus.DELETE (successfully completed). */ class FileStatusHandler { private static Logger logger = LoggerFactory.getLogger(FileStatusHandler.class); public static void onStatusChange(StreamContext context, PendingFile pendingFile, FileStatus streamStatus) throws IOException { if (FileStatus.Action.STREAM == streamStatus.getAction()) { // file needs to be restreamed logger.warn("Streaming of file {} from {} failed: requesting a retry.", pendingFile, context); MessagingService.instance.sendOneWay(streamStatus.makeStreamStatusMessage(), context.host); return; } assert FileStatus.Action.DELETE == streamStatus.getAction() : "Unknown stream action: " + streamStatus.getAction(); addSSTable(pendingFile, context); // send a StreamStatus message telling the source node it can delete this file if (logger.isDebugEnabled()) logger.debug("Sending a streaming finished message for {} to {}", pendingFile, context); MessagingService.instance.sendOneWay(streamStatus.makeStreamStatusMessage(), context.host); } public static void addSSTable(PendingFile pendingFile, StreamContext context) { // file was successfully streamed Descriptor desc = pendingFile.desc; try { // right now we only need to do this differently for AES operations IRecoveryProcessor rp = pendingFile.type == OperationType.AES ? new AESRecoveryProcessor(context.host) : IndexRecoveryProcessor.instance(); SSTableReader sstable = SSTableWriter.recoverAndOpen(pendingFile.desc, rp); Table.open(desc.ksname).getColumnFamilyStore(desc.cfname).addSSTable(sstable); logger.info("Streaming added " + sstable); } catch (IOException e) { logger.error("Failed adding {}", pendingFile, e); throw new RuntimeException("Not able to add streamed file " + pendingFile.getFilename(), e); } } }