001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.master.snapshot; 019 020import java.io.IOException; 021import org.apache.hadoop.hbase.exceptions.DeserializationException; 022import org.apache.hadoop.hbase.master.BooleanStateStore; 023import org.apache.hadoop.hbase.master.region.MasterRegion; 024import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 025import org.apache.yetus.audience.InterfaceAudience; 026import org.apache.zookeeper.KeeperException; 027 028import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 029import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotCleanupProtos; 030 031/** 032 * Store the snapshot cleanup enabled state. 033 */ 034@InterfaceAudience.Private 035public class SnapshotCleanupStateStore extends BooleanStateStore { 036 037 public static final String STATE_NAME = "snapshot_cleanup_enabled"; 038 039 @SuppressWarnings("deprecation") 040 public SnapshotCleanupStateStore(MasterRegion masterRegion, ZKWatcher watcher) 041 throws IOException, KeeperException, DeserializationException { 042 super(masterRegion, STATE_NAME, watcher, watcher.getZNodePaths().snapshotCleanupZNode); 043 } 044 045 @Override 046 protected byte[] toByteArray(boolean isSnapshotCleanupEnabled) { 047 SnapshotCleanupProtos.SnapshotCleanupState.Builder builder = 048 SnapshotCleanupProtos.SnapshotCleanupState.newBuilder(); 049 builder.setSnapshotCleanupEnabled(isSnapshotCleanupEnabled); 050 return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); 051 } 052 053 @Override 054 protected boolean parseFrom(byte[] pbBytes) throws DeserializationException { 055 ProtobufUtil.expectPBMagicPrefix(pbBytes); 056 SnapshotCleanupProtos.SnapshotCleanupState.Builder builder = 057 SnapshotCleanupProtos.SnapshotCleanupState.newBuilder(); 058 try { 059 int magicLen = ProtobufUtil.lengthOfPBMagic(); 060 ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen); 061 } catch (IOException e) { 062 throw new DeserializationException(e); 063 } 064 return builder.build().getSnapshotCleanupEnabled(); 065 } 066}