/** * Parses an HRegionInfo instance from the passed in stream. Presumes the HRegionInfo was * serialized to the stream with {@link #toDelimitedByteArray()} * * @param in * @return An instance of HRegionInfo. * @throws IOException */ public static HRegionInfo parseFrom(final DataInputStream in) throws IOException { // I need to be able to move back in the stream if this is not a pb serialization so I can // do the Writable decoding instead. int pblen = ProtobufUtil.lengthOfPBMagic(); byte[] pbuf = new byte[pblen]; if (in.markSupported()) { // read it with mark() in.mark(pblen); } int read = in.read(pbuf); // assumption: if Writable serialization, it should be longer than pblen. if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen); if (ProtobufUtil.isPBMagicPrefix(pbuf)) { return convert(HBaseProtos.RegionInfo.parseDelimitedFrom(in)); } else { // Presume Writables. Need to reset the stream since it didn't start w/ pb. if (in.markSupported()) { in.reset(); HRegionInfo hri = new HRegionInfo(); hri.readFields(in); return hri; } else { // we cannot use BufferedInputStream, it consumes more than we read from the underlying IS ByteArrayInputStream bais = new ByteArrayInputStream(pbuf); SequenceInputStream sis = new SequenceInputStream(bais, in); // concatenate input streams HRegionInfo hri = new HRegionInfo(); hri.readFields(new DataInputStream(sis)); return hri; } } }
/** * Populate this instance with what we find on the passed in <code>in</code> stream. Can * deserialize protobuf of old Writables format. * * @param in * @throws IOException * @see #write(DataOutputStream) */ void read(final DataInputStream in) throws IOException { // This code is tested over in TestHFileReaderV1 where we read an old hfile w/ this new code. int pblen = ProtobufUtil.lengthOfPBMagic(); byte[] pbuf = new byte[pblen]; if (in.markSupported()) in.mark(pblen); int read = in.read(pbuf); if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen); if (ProtobufUtil.isPBMagicPrefix(pbuf)) { parsePB(HFileProtos.FileInfoProto.parseDelimitedFrom(in)); } else { if (in.markSupported()) { in.reset(); parseWritable(in); } else { // We cannot use BufferedInputStream, it consumes more than we read from the underlying IS ByteArrayInputStream bais = new ByteArrayInputStream(pbuf); SequenceInputStream sis = new SequenceInputStream(bais, in); // Concatenate input streams // TODO: Am I leaking anything here wrapping the passed in stream? We are not calling // close on the wrapped // streams but they should be let go after we leave this context? I see that we keep a // reference to the // passed in inputstream but since we no longer have a reference to this after we leave, // we should be ok. parseWritable(new DataInputStream(sis)); } } }
/** * Kick off a new sub-procedure on the listener with the data stored in the passed znode. * * <p>Will attempt to create the same procedure multiple times if an procedure znode with the same * name is created. It is left up the coordinator to ensure this doesn't occur. * * @param path full path to the znode for the procedure to start */ private synchronized void startNewSubprocedure(String path) { LOG.debug("Found procedure znode: " + path); String opName = ZKUtil.getNodeName(path); // start watching for an abort notification for the procedure String abortZNode = zkController.getAbortZNode(opName); try { if (ZKUtil.watchAndCheckExists(zkController.getWatcher(), abortZNode)) { LOG.debug("Not starting:" + opName + " because we already have an abort notification."); return; } } catch (KeeperException e) { member.controllerConnectionFailure( "Failed to get the abort znode (" + abortZNode + ") for procedure :" + opName, e, opName); return; } // get the data for the procedure Subprocedure subproc = null; try { byte[] data = ZKUtil.getData(zkController.getWatcher(), path); if (!ProtobufUtil.isPBMagicPrefix(data)) { String msg = "Data in for starting procuedure " + opName + " is illegally formatted (no pb magic). " + "Killing the procedure: " + Bytes.toString(data); LOG.error(msg); throw new IllegalArgumentException(msg); } LOG.debug("start proc data length is " + data.length); data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length); LOG.debug("Found data for znode:" + path); subproc = member.createSubprocedure(opName, data); member.submitSubprocedure(subproc); } catch (IllegalArgumentException iae) { LOG.error("Illegal argument exception", iae); sendMemberAborted(subproc, new ForeignException(getMemberName(), iae)); } catch (IllegalStateException ise) { LOG.error("Illegal state exception ", ise); sendMemberAborted(subproc, new ForeignException(getMemberName(), ise)); } catch (KeeperException e) { member.controllerConnectionFailure( "Failed to get data for new procedure:" + opName, e, opName); } catch (InterruptedException e) { member.controllerConnectionFailure( "Failed to get data for new procedure:" + opName, e, opName); Thread.currentThread().interrupt(); } }
/** * Pass along the found abort notification to the listener * * @param abortZNode full znode path to the failed procedure information */ protected void abort(String abortZNode) { LOG.debug("Aborting procedure member for znode " + abortZNode); String opName = ZKUtil.getNodeName(abortZNode); try { byte[] data = ZKUtil.getData(zkController.getWatcher(), abortZNode); // figure out the data we need to pass ForeignException ee; try { if (data == null || data.length == 0) { // ignore return; } else if (!ProtobufUtil.isPBMagicPrefix(data)) { String msg = "Illegally formatted data in abort node for proc " + opName + ". Killing the procedure."; LOG.error(msg); // we got a remote exception, but we can't describe it so just return exn from here ee = new ForeignException(getMemberName(), new IllegalArgumentException(msg)); } else { data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length); ee = ForeignException.deserialize(data); } } catch (InvalidProtocolBufferException e) { LOG.warn( "Got an error notification for op:" + opName + " but we can't read the information. Killing the procedure."); // we got a remote exception, but we can't describe it so just return exn from here ee = new ForeignException(getMemberName(), e); } this.member.receiveAbortProcedure(opName, ee); } catch (KeeperException e) { member.controllerConnectionFailure( "Failed to get data for abort znode:" + abortZNode + zkController.getAbortZnode(), e, opName); } catch (InterruptedException e) { LOG.warn("abort already in progress", e); Thread.currentThread().interrupt(); } }
/** * @param bytes Content of a peer znode. * @return ClusterKey parsed from the passed bytes. * @throws DeserializationException */ private static String parsePeerFrom(final byte[] bytes) throws DeserializationException { if (ProtobufUtil.isPBMagicPrefix(bytes)) { int pblen = ProtobufUtil.lengthOfPBMagic(); ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder(); ZooKeeperProtos.ReplicationPeer peer; try { peer = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build(); } catch (InvalidProtocolBufferException e) { throw new DeserializationException(e); } return peer.getClusterkey(); } else { if (bytes.length > 0) { return Bytes.toString(bytes); } return ""; } }
/** * @param bytes A pb RegionInfo serialized with a pb magic prefix. * @param offset starting point in the byte array * @param len length to read on the byte array * @return A deserialized {@link HRegionInfo} * @throws DeserializationException * @see #toByteArray() */ public static HRegionInfo parseFrom(final byte[] bytes, int offset, int len) throws DeserializationException { if (ProtobufUtil.isPBMagicPrefix(bytes, offset, len)) { int pblen = ProtobufUtil.lengthOfPBMagic(); try { HBaseProtos.RegionInfo ri = HBaseProtos.RegionInfo.newBuilder() .mergeFrom(bytes, pblen + offset, len - pblen) .build(); return convert(ri); } catch (InvalidProtocolBufferException e) { throw new DeserializationException(e); } } else { try { HRegionInfo hri = new HRegionInfo(); hri.readFields(bytes, offset, len); return hri; } catch (IOException e) { throw new DeserializationException(e); } } }
/** * Load the meta region state from the meta server ZNode. * * @param zkw * @param replicaId * @return regionstate * @throws KeeperException */ public static RegionState getMetaRegionState(ZooKeeperWatcher zkw, int replicaId) throws KeeperException { RegionState.State state = RegionState.State.OPEN; ServerName serverName = null; try { byte[] data = ZKUtil.getData(zkw, zkw.getZNodeForReplica(replicaId)); if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) { try { int prefixLen = ProtobufUtil.lengthOfPBMagic(); ZooKeeperProtos.MetaRegionServer rl = ZooKeeperProtos.MetaRegionServer.PARSER.parseFrom( data, prefixLen, data.length - prefixLen); if (rl.hasState()) { state = RegionState.State.convert(rl.getState()); } HBaseProtos.ServerName sn = rl.getServer(); serverName = ServerName.valueOf(sn.getHostName(), sn.getPort(), sn.getStartCode()); } catch (InvalidProtocolBufferException e) { throw new DeserializationException("Unable to parse meta region location"); } } else { // old style of meta region location? serverName = ServerName.parseFrom(data); } } catch (DeserializationException e) { throw ZKUtil.convert(e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } if (serverName == null) { state = RegionState.State.OFFLINE; } return new RegionState( RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId), state, serverName); }