|
@@ -408,87 +408,182 @@ void removeFileEmptyScope(const CDfsLogicalFileName &dlfn,unsigned timeout)
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
-class CFileConnectLock
|
|
|
+class CFileLockBase
|
|
|
{
|
|
|
- CConnectLock *lock;
|
|
|
- bool attronly;
|
|
|
+ IRemoteConnection *conn;
|
|
|
+protected:
|
|
|
+ Owned<IRemoteConnection> lock;
|
|
|
+
|
|
|
+ bool init(StringBuffer &lockPath, unsigned mode, IRemoteConnection *_conn, unsigned timeout, const char *msg)
|
|
|
+ {
|
|
|
+ conn = NULL;
|
|
|
+ lock.clear();
|
|
|
+ CTimeMon tm(timeout);
|
|
|
+ loop
|
|
|
+ {
|
|
|
+ try
|
|
|
+ {
|
|
|
+ lock.setown(querySDS().connect(lockPath.str(), myProcessSession(), mode, timeout>60000 ? 60000 : timeout));
|
|
|
+ if (lock.get())
|
|
|
+ {
|
|
|
+ conn = _conn;
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ catch (ISDSException *e)
|
|
|
+ {
|
|
|
+ unsigned remaining;
|
|
|
+ if (SDSExcpt_LockTimeout != e->errorCode() || tm.timedout(&remaining))
|
|
|
+ throw;
|
|
|
+ WARNLOG("CFileAttrLockBase(%s) blocked for %ds", msg, tm.elapsed()/1000);
|
|
|
+ e->Release();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
public:
|
|
|
- CFileConnectLock(bool _attronly=false)
|
|
|
+ CFileLockBase()
|
|
|
{
|
|
|
- lock = NULL;
|
|
|
- attronly = _attronly;
|
|
|
+ conn = NULL;
|
|
|
}
|
|
|
- CFileConnectLock(const char *caller,const CDfsLogicalFileName &lname,DfsXmlBranchKind bkind,bool write,bool preload,bool hold,unsigned timeout,bool _attronly=false)
|
|
|
+ ~CFileLockBase()
|
|
|
{
|
|
|
- lock = NULL;
|
|
|
- attronly = _attronly;
|
|
|
- init(caller,lname,bkind,write,preload,hold,timeout);
|
|
|
+ // if conn provided, 'lock' was just a surrogate for the owner connection, commit now to conn if write lock
|
|
|
+ if (conn && lock && write)
|
|
|
+ conn->commit();
|
|
|
}
|
|
|
- ~CFileConnectLock()
|
|
|
+ IRemoteConnection *detach()
|
|
|
{
|
|
|
- delete lock;
|
|
|
+ return lock.getClear();
|
|
|
}
|
|
|
-
|
|
|
- bool init(const char *caller,const CDfsLogicalFileName &lname,DfsXmlBranchKind bkind,bool write,bool preload,bool hold,unsigned timeout)
|
|
|
+ void clear()
|
|
|
{
|
|
|
- kill();
|
|
|
- StringBuffer query;
|
|
|
- lname.makeFullnameQuery(query,bkind,true);
|
|
|
- if (attronly)
|
|
|
- query.append("/Attr");
|
|
|
- lock = new CConnectLock(caller,query.str(),write,preload,hold,timeout);
|
|
|
- return lock->conn.get()!=NULL;
|
|
|
+ lock.clear();
|
|
|
+ conn = NULL;
|
|
|
+ }
|
|
|
+ IPropertyTree *queryRoot() const
|
|
|
+ {
|
|
|
+ return lock.get() ? lock->queryRoot() : NULL;
|
|
|
}
|
|
|
+};
|
|
|
|
|
|
- bool initany(const char *caller,const CDfsLogicalFileName &lname,DfsXmlBranchKind &bkind,bool write,bool preload,bool hold,unsigned timeout)
|
|
|
+class CFileLock : protected CFileLockBase
|
|
|
+{
|
|
|
+protected:
|
|
|
+ DfsXmlBranchKind kind;
|
|
|
+public:
|
|
|
+ CFileLock()
|
|
|
{
|
|
|
- if (init(caller, lname, DXB_File, write, preload, hold, timeout))
|
|
|
+ kind = DXB_Internal;
|
|
|
+ }
|
|
|
+ bool init(const CDfsLogicalFileName &logicalName, DfsXmlBranchKind bkind, unsigned mode, unsigned timeout, const char *msg)
|
|
|
+ {
|
|
|
+ StringBuffer lockPath;
|
|
|
+ logicalName.makeFullnameQuery(lockPath, bkind, true);
|
|
|
+ if (CFileLockBase::init(lockPath, mode, NULL, timeout, msg))
|
|
|
{
|
|
|
- bkind = DXB_File;
|
|
|
+ kind = bkind;
|
|
|
return true;
|
|
|
}
|
|
|
- if (init(caller, lname, DXB_SuperFile, write, preload, hold, timeout))
|
|
|
+ kind = DXB_Internal;
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ bool init(const CDfsLogicalFileName &logicalName, unsigned mode, unsigned timeout, const char *msg)
|
|
|
+ {
|
|
|
+ StringBuffer lockPath;
|
|
|
+ logicalName.makeFullnameQuery(lockPath, DXB_File, true);
|
|
|
+ if (CFileLockBase::init(lockPath, mode, NULL, timeout, msg))
|
|
|
{
|
|
|
- bkind = DXB_SuperFile;
|
|
|
+ kind = DXB_File;
|
|
|
return true;
|
|
|
}
|
|
|
+ // try super
|
|
|
+ logicalName.makeFullnameQuery(lockPath.clear(), DXB_SuperFile, true);
|
|
|
+ if (CFileLockBase::init(lockPath, mode, NULL, timeout, msg))
|
|
|
+ {
|
|
|
+ kind = DXB_SuperFile;
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ kind = DXB_Internal;
|
|
|
return false;
|
|
|
}
|
|
|
-
|
|
|
- IRemoteConnection *detach()
|
|
|
+ IRemoteConnection *detach() { return CFileLockBase::detach(); }
|
|
|
+ IPropertyTree *queryRoot() const { return CFileLockBase::queryRoot(); }
|
|
|
+ IRemoteConnection *queryConnection() const
|
|
|
{
|
|
|
- return lock?lock->detach():NULL;
|
|
|
+ return lock;
|
|
|
}
|
|
|
-
|
|
|
- IRemoteConnection *conn()
|
|
|
+ void clear()
|
|
|
{
|
|
|
- return lock?lock->conn:NULL;
|
|
|
+ CFileLockBase::clear();
|
|
|
+ kind = DXB_Internal;
|
|
|
}
|
|
|
+ DfsXmlBranchKind getKind() const { return kind; }
|
|
|
+};
|
|
|
|
|
|
- IPropertyTree *queryRoot() const
|
|
|
+class CFileSubLock : protected CFileLockBase
|
|
|
+{
|
|
|
+public:
|
|
|
+ bool init(const CDfsLogicalFileName &logicalName, DfsXmlBranchKind bkind, unsigned mode, const char *subLock, IRemoteConnection *conn, unsigned timeout, const char *msg)
|
|
|
{
|
|
|
- return (lock&&lock->conn.get())?lock->conn->queryRoot():NULL;
|
|
|
+ StringBuffer lockPath;
|
|
|
+ logicalName.makeFullnameQuery(lockPath, bkind, true);
|
|
|
+ lockPath.appendf("/%s", subLock);
|
|
|
+ return CFileLockBase::init(lockPath, mode, conn, timeout, msg);
|
|
|
+ }
|
|
|
+ bool init(const CDfsLogicalFileName &logicalName, unsigned mode, const char *subLock, IRemoteConnection *conn, unsigned timeout, const char *msg)
|
|
|
+ {
|
|
|
+ StringBuffer lockPath;
|
|
|
+ logicalName.makeFullnameQuery(lockPath, DXB_File, true);
|
|
|
+ lockPath.appendf("/%s", subLock);
|
|
|
+ if (CFileLockBase::init(lockPath, mode, conn, timeout, msg))
|
|
|
+ return true;
|
|
|
+ // try super
|
|
|
+ logicalName.makeFullnameQuery(lockPath.clear(), DXB_SuperFile, true);
|
|
|
+ return CFileLockBase::init(lockPath, mode, conn, timeout, msg);
|
|
|
}
|
|
|
+};
|
|
|
|
|
|
- void remove()
|
|
|
+
|
|
|
+class CFileAttrLock : protected CFileSubLock
|
|
|
+{
|
|
|
+public:
|
|
|
+ bool init(const CDfsLogicalFileName &logicalName, DfsXmlBranchKind bkind, unsigned mode, IRemoteConnection *conn, unsigned timeout, const char *msg)
|
|
|
{
|
|
|
- if (lock&&lock->conn.get()) {
|
|
|
-#ifdef TRACE_LOCKS
|
|
|
- PROGLOG("CFileConnectLock:remove%s",attronly?" Attr":"");
|
|
|
- LogRemoteConn(lock->conn);
|
|
|
-#endif
|
|
|
- lock->conn->close(true);
|
|
|
- }
|
|
|
+ return CFileSubLock::init(logicalName, bkind, mode, "Attr", conn, timeout, msg);
|
|
|
}
|
|
|
+ bool init(const CDfsLogicalFileName &logicalName, unsigned mode, IRemoteConnection *conn, unsigned timeout, const char *msg)
|
|
|
+ {
|
|
|
+ return CFileSubLock::init(logicalName, mode, "Attr", conn, timeout, msg);
|
|
|
+ }
|
|
|
+ IPropertyTree *queryRoot() const { return CFileSubLock::queryRoot(); }
|
|
|
+};
|
|
|
|
|
|
- void kill()
|
|
|
+class CFileLockCompound : protected CFileLockBase
|
|
|
+{
|
|
|
+public:
|
|
|
+ bool init(const CDfsLogicalFileName &logicalName, unsigned mode, IRemoteConnection *conn, const char *subLock, unsigned timeout, const char *msg)
|
|
|
{
|
|
|
- delete lock;
|
|
|
- lock = NULL;
|
|
|
+ StringBuffer lockPath;
|
|
|
+ if (subLock)
|
|
|
+ lockPath.appendf("/_Locks/%s/", subLock);
|
|
|
+ logicalName.makeXPathLName(lockPath);
|
|
|
+ return CFileLockBase::init(lockPath, mode, conn, timeout, msg);
|
|
|
}
|
|
|
};
|
|
|
|
|
|
+class CFileSuperOwnerLock : protected CFileLockCompound
|
|
|
+{
|
|
|
+public:
|
|
|
+ bool init(const CDfsLogicalFileName &logicalName, IRemoteConnection *conn, unsigned timeout, const char *msg)
|
|
|
+ {
|
|
|
+ return CFileLockCompound::init(logicalName, RTM_CREATE_QUERY | RTM_LOCK_WRITE | RTM_DELETE_ON_DISCONNECT, conn, "SuperOwnerLock", timeout, msg);
|
|
|
+ }
|
|
|
+ IRemoteConnection *detach()
|
|
|
+ {
|
|
|
+ return CFileLockCompound::detach();
|
|
|
+ }
|
|
|
+};
|
|
|
|
|
|
class CScopeConnectLock
|
|
|
{
|
|
@@ -885,10 +980,10 @@ public:
|
|
|
redirection.setown(createDFSredirection());
|
|
|
}
|
|
|
|
|
|
- IDistributedFile *dolookup(CDfsLogicalFileName &logicalname, IUserDescriptor *user, bool writeattr, bool hold, IDistributedFileTransaction *transaction, unsigned timeout);
|
|
|
+ IDistributedFile *dolookup(CDfsLogicalFileName &logicalname, IUserDescriptor *user, bool writeattr, bool hold, bool lockSuperOwner, IDistributedFileTransaction *transaction, unsigned timeout);
|
|
|
|
|
|
- IDistributedFile *lookup(const char *_logicalname, IUserDescriptor *user, bool writeattr, bool hold, IDistributedFileTransaction *transaction, unsigned timeout);
|
|
|
- IDistributedFile *lookup(CDfsLogicalFileName &logicalname, IUserDescriptor *user, bool writeattr, bool hold, IDistributedFileTransaction *transaction, unsigned timeout);
|
|
|
+ IDistributedFile *lookup(const char *_logicalname, IUserDescriptor *user, bool writeattr, bool hold, bool lockSuperOwner, IDistributedFileTransaction *transaction, unsigned timeout);
|
|
|
+ IDistributedFile *lookup(CDfsLogicalFileName &logicalname, IUserDescriptor *user, bool writeattr, bool hold, bool lockSuperOwner, IDistributedFileTransaction *transaction, unsigned timeout);
|
|
|
|
|
|
/* createNew always creates an unnamed unattached distributed file
|
|
|
* The caller must associated it with a name and credentials when it is attached (attach())
|
|
@@ -1206,7 +1301,7 @@ public:
|
|
|
ThrowStringException(-1, "Logical Name fails for removal on %s", lfn.get());
|
|
|
|
|
|
// Transaction files have already been unlocked at this point, delete all remaining files
|
|
|
- Owned<IDistributedFile> file = queryDistributedFileDirectory().lookup(lfn, user, true, false, NULL, SDS_SUB_LOCK_TIMEOUT);
|
|
|
+ Owned<IDistributedFile> file = queryDistributedFileDirectory().lookup(lfn, user, true, false, true, NULL, SDS_SUB_LOCK_TIMEOUT);
|
|
|
if (!file.get())
|
|
|
return;
|
|
|
StringBuffer reason;
|
|
@@ -1597,7 +1692,7 @@ public:
|
|
|
return LINK(ret);
|
|
|
else
|
|
|
{
|
|
|
- ret = queryDistributedFileDirectory().lookup(name, udesc, false, false, this, timeout);
|
|
|
+ ret = queryDistributedFileDirectory().lookup(name, udesc, false, false, false, this, timeout);
|
|
|
if (ret)
|
|
|
queryCreate(name, ret, true);
|
|
|
return ret;
|
|
@@ -2339,17 +2434,20 @@ class CDistributedSuperFileIterator: public CDistributedFileIteratorBase<IDistri
|
|
|
Linked<IUserDescriptor> udesc;
|
|
|
Linked<IDistributedFileTransaction> transaction;
|
|
|
Owned<IDistributedSuperFile> cur;
|
|
|
+ Linked<IDistributedFile> owner;
|
|
|
|
|
|
public:
|
|
|
- CDistributedSuperFileIterator(CDistributedFileDirectory *_parent,IPropertyTree *root,IUserDescriptor *user, IDistributedFileTransaction *_transaction)
|
|
|
- : transaction(_transaction)
|
|
|
+ CDistributedSuperFileIterator(IDistributedFile *_owner, CDistributedFileDirectory *_parent,IPropertyTree *root,IUserDescriptor *user, IDistributedFileTransaction *_transaction)
|
|
|
+ : owner(_owner), transaction(_transaction)
|
|
|
{
|
|
|
setUserDescriptor(udesc,user);
|
|
|
parent = _parent;
|
|
|
- if (root) {
|
|
|
+ if (root)
|
|
|
+ {
|
|
|
Owned<IPropertyTreeIterator> iter = root->getElements("SuperOwner");
|
|
|
StringBuffer pname;
|
|
|
- ForEach(*iter) {
|
|
|
+ ForEach(*iter)
|
|
|
+ {
|
|
|
iter->query().getProp("@name",pname.clear());
|
|
|
if (pname.length())
|
|
|
list.append(* new StringAttrItem(pname.str()));
|
|
@@ -2479,6 +2577,7 @@ protected:
|
|
|
Linked<IUserDescriptor> udesc;
|
|
|
unsigned defaultTimeout;
|
|
|
bool dirty;
|
|
|
+ Owned<IRemoteConnection> superOwnerLock;
|
|
|
public:
|
|
|
|
|
|
IPropertyTree *queryRoot() { return root; }
|
|
@@ -2497,6 +2596,11 @@ public:
|
|
|
root.clear();
|
|
|
}
|
|
|
|
|
|
+ void setSuperOwnerLock(IRemoteConnection *_superOwnerLock)
|
|
|
+ {
|
|
|
+ superOwnerLock.setown(_superOwnerLock);
|
|
|
+ }
|
|
|
+
|
|
|
unsigned setPropLockCount(unsigned _propLockCount)
|
|
|
{
|
|
|
unsigned prevPropLockCount = proplockcount;
|
|
@@ -2741,6 +2845,9 @@ public:
|
|
|
virtual bool isSubFile()
|
|
|
{
|
|
|
CriticalBlock block(sect);
|
|
|
+ // JCSMORE - this method should probably be removed, only file view uses it.
|
|
|
+ if (conn) // SuperOwner, could have changed whilst locked, ensure refreshed
|
|
|
+ conn->reload("SuperOwner");
|
|
|
return root&&root->hasProp("SuperOwner[1]");
|
|
|
}
|
|
|
|
|
@@ -2769,7 +2876,7 @@ public:
|
|
|
virtual IDistributedSuperFileIterator *getOwningSuperFiles(IDistributedFileTransaction *_transaction)
|
|
|
{
|
|
|
CriticalBlock block(sect);
|
|
|
- return new CDistributedSuperFileIterator(parent,root,udesc,_transaction);
|
|
|
+ return new CDistributedSuperFileIterator(this,parent,root,udesc,_transaction);
|
|
|
}
|
|
|
|
|
|
virtual void checkFormatAttr(IDistributedFile *sub, const char* exprefix="")
|
|
@@ -2857,7 +2964,11 @@ public:
|
|
|
{
|
|
|
if (!superfile||!*superfile)
|
|
|
return;
|
|
|
- if (conn) {
|
|
|
+ if (conn)
|
|
|
+ {
|
|
|
+ CFileSuperOwnerLock attrLock;
|
|
|
+ if (0 == proplockcount)
|
|
|
+ verifyex(attrLock.init(logicalName, conn, defaultTimeout, "CDistributedFile::linkSuperOwner"));
|
|
|
Owned<IPropertyTree> t = getNamedPropTree(root,"SuperOwner","@name",superfile,false);
|
|
|
if (t && !link)
|
|
|
root->removeTree(t);
|
|
@@ -3075,6 +3186,9 @@ protected:
|
|
|
throw MakeStringException(-1,"detach: %s", reason.str());
|
|
|
}
|
|
|
// detach this IDistributeFile
|
|
|
+
|
|
|
+ /* JCSMORE - In 'removeFile=true' case, this should really delete before release exclusive lock.
|
|
|
+ */
|
|
|
writeLock.clear();
|
|
|
root.setown(closeConnection(removeFile));
|
|
|
// NB: The file is now unlocked
|
|
@@ -3730,7 +3844,8 @@ public:
|
|
|
parent->addEntry(logicalName,root.getClear(),false,false);
|
|
|
killParts();
|
|
|
clusters.kill();
|
|
|
- CFileConnectLock fcl("CDistributedFile::attach",logicalName,DXB_File, false, false, false, defaultTimeout);
|
|
|
+ CFileLock fcl;
|
|
|
+ verifyex(fcl.init(logicalName, DXB_File, RTM_LOCK_READ, defaultTimeout, "CDistributedFile::attach"));
|
|
|
conn.setown(fcl.detach());
|
|
|
root.setown(conn->getRoot());
|
|
|
root->queryBranch("."); // load branch
|
|
@@ -4338,15 +4453,9 @@ public:
|
|
|
parent->setFileAccessed(logicalName,udesc,dt);
|
|
|
else
|
|
|
{
|
|
|
- CFileConnectLock fconnattrlock(true);
|
|
|
- bool performedLock = false;
|
|
|
- if (conn && !proplockcount) // NB: If locked, already have exclusive access to file
|
|
|
- {
|
|
|
- DfsXmlBranchKind bkind;
|
|
|
- if (!fconnattrlock.initany("CDistributedFile::setAccessedTime", logicalName, bkind, true, false, false, defaultTimeout))
|
|
|
- return; // timeout will raise exception
|
|
|
- performedLock = true;
|
|
|
- }
|
|
|
+ CFileAttrLock attrLock;
|
|
|
+ if (0 == proplockcount && conn)
|
|
|
+ verifyex(attrLock.init(logicalName, DXB_File, RTM_LOCK_WRITE, conn, defaultTimeout, "CDistributedFile::setAccessedTime"));
|
|
|
if (dt.isNull())
|
|
|
queryAttributes().removeProp("@accessed");
|
|
|
else
|
|
@@ -4354,12 +4463,10 @@ public:
|
|
|
StringBuffer str;
|
|
|
queryAttributes().setProp("@accessed",dt.getString(str).str());
|
|
|
}
|
|
|
- if (performedLock)
|
|
|
- conn->commit();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void setAccessed()
|
|
|
+ void setAccessed()
|
|
|
{
|
|
|
CDateTime dt;
|
|
|
dt.setNow();
|
|
@@ -4434,7 +4541,6 @@ class CDistributedSuperFile: public CDistributedFileBase<IDistributedSuperFile>
|
|
|
}
|
|
|
// Try to lock all files
|
|
|
addFileLock(parent);
|
|
|
- addFileLock(sub);
|
|
|
if (lock())
|
|
|
{
|
|
|
transaction->noteAddSubFile(parent, parentlname, sub);
|
|
@@ -4500,7 +4606,7 @@ class CDistributedSuperFile: public CDistributedFileBase<IDistributedSuperFile>
|
|
|
}
|
|
|
// Try to lock all files
|
|
|
addFileLock(parent);
|
|
|
- if (sub)
|
|
|
+ if (sub && remsub) // NB: I only need to lock (for exclusivity, if going to delete
|
|
|
addFileLock(sub);
|
|
|
if (lock())
|
|
|
{
|
|
@@ -4708,6 +4814,10 @@ protected:
|
|
|
|
|
|
void clearSuperOwners(unsigned timeoutMs)
|
|
|
{
|
|
|
+ /* JCSMORE - Why on earth is this doing this way?
|
|
|
+ * We are in a super file, we already have [read] locks to sub files (in 'subfiles' array)
|
|
|
+ * This should iterate through those and call unlinkSubFile I think.
|
|
|
+ */
|
|
|
Owned<IPropertyTreeIterator> iter = root->getElements("SubFile");
|
|
|
StringBuffer oquery;
|
|
|
oquery.append("SuperOwner[@name=\"").append(logicalName.get()).append("\"]");
|
|
@@ -4719,10 +4829,10 @@ protected:
|
|
|
{
|
|
|
CDfsLogicalFileName subfn;
|
|
|
subfn.set(name);
|
|
|
- CFileConnectLock fconnlockSub;
|
|
|
- DfsXmlBranchKind subbkind;
|
|
|
+ CFileLock fconnlockSub;
|
|
|
+ // JCSMORE - this is really not right, but consistent with previous version
|
|
|
// MORE: Use CDistributedSuperFile::linkSuperOwner(false) - ie. unlink
|
|
|
- if (fconnlockSub.initany("CDelayedDelete::doRemoveEntry", subfn, subbkind, false, false, false, timeoutMs))
|
|
|
+ if (fconnlockSub.init(subfn, RTM_LOCK_READ, timeoutMs, "CDistributedFile::doRemoveEntry"))
|
|
|
{
|
|
|
IPropertyTree *subfroot = fconnlockSub.queryRoot();
|
|
|
if (subfroot)
|
|
@@ -4802,7 +4912,7 @@ protected:
|
|
|
IPropertyTree &sub = *(orderedSubFiles[f]);
|
|
|
sub.getProp("@name",subname.clear());
|
|
|
Owned<IDistributedFile> subfile;
|
|
|
- subfile.setown(transaction?transaction->lookupFile(subname.str(),timeout):parent->lookup(subname.str(), udesc, false, false, transaction, timeout));
|
|
|
+ subfile.setown(transaction?transaction->lookupFile(subname.str(),timeout):parent->lookup(subname.str(), udesc, false, false, false, transaction, timeout));
|
|
|
if (!subfile.get())
|
|
|
subfile.setown(transaction?transaction->lookupSuperFile(subname.str(),timeout):parent->lookupSuperFile(subname.str(),udesc,transaction,timeout));
|
|
|
|
|
@@ -4945,10 +5055,9 @@ protected:
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void linkSubFile(unsigned pos,IDistributedFileTransactionExt *transaction,bool link=true)
|
|
|
+ void linkSubFile(unsigned pos, bool link=true)
|
|
|
{
|
|
|
IDistributedFile *subfile = &subfiles.item(pos);
|
|
|
- DistributedFilePropertyLock lock(subfile);
|
|
|
IDistributedSuperFile *ssub = subfile->querySuperFile();
|
|
|
if (ssub) {
|
|
|
CDistributedSuperFile *cdsuper = QUERYINTERFACE(ssub,CDistributedSuperFile);
|
|
@@ -4958,13 +5067,11 @@ protected:
|
|
|
CDistributedFile *cdfile = QUERYINTERFACE(subfile,CDistributedFile);
|
|
|
cdfile->linkSuperOwner(queryLogicalName(),link);
|
|
|
}
|
|
|
- if (!transaction || !transaction->active())
|
|
|
- lock.commit();
|
|
|
}
|
|
|
|
|
|
- void unlinkSubFile(unsigned pos,IDistributedFileTransactionExt *transaction)
|
|
|
+ void unlinkSubFile(unsigned pos)
|
|
|
{
|
|
|
- linkSubFile(pos, transaction, false);
|
|
|
+ linkSubFile(pos, false);
|
|
|
}
|
|
|
|
|
|
void checkSubFormatAttr(IDistributedFile *sub, const char* exprefix="")
|
|
@@ -5303,7 +5410,8 @@ public:
|
|
|
checkLogicalName(logicalName,user,true,true,false,"attach");
|
|
|
parent->addEntry(logicalName,root.getClear(),true,false);
|
|
|
conn.clear();
|
|
|
- CFileConnectLock fcl("CDistributedSuperFile::attach",logicalName,DXB_SuperFile, false, false, false, defaultTimeout);
|
|
|
+ CFileLock fcl;
|
|
|
+ verifyex(fcl.init(logicalName, DXB_SuperFile, RTM_LOCK_READ, defaultTimeout, "CDistributedSuperFile::attach"));
|
|
|
conn.setown(fcl.detach());
|
|
|
assertex(conn.get()); // must have been attached
|
|
|
root.setown(conn->getRoot());
|
|
@@ -5320,6 +5428,16 @@ public:
|
|
|
subfiles.kill();
|
|
|
|
|
|
// Remove from SDS
|
|
|
+
|
|
|
+ /* JCSMORE - this looks very kludgy...
|
|
|
+ * We have readlock, this code is doing
|
|
|
+ * 1) change to write lock (not using lockProperties or DistributedFilePropertyLock to do so) [using CFileChangeWriteLock]
|
|
|
+ * CFileChangeWriteLock doesn't preserve lock mode quite right.. (see 'newMode')
|
|
|
+ * 2) manually deleting SuperOwner from subfiles (in clearSuperOwners)
|
|
|
+ * 3) Using the connection to delete the SuperFile from Dali (clones to 'root' in process)
|
|
|
+ * 4) ~CFileChangeWriteLock() [writeLock.clear()], restores read lock from write to read
|
|
|
+ * 5) updateFS (housekeeping of empty scopes, relationships) - ok
|
|
|
+ */
|
|
|
CFileChangeWriteLock writeLock(conn, timeoutMs);
|
|
|
clearSuperOwners(timeoutMs);
|
|
|
writeLock.clear();
|
|
@@ -5714,7 +5832,7 @@ private:
|
|
|
addItem(pos,sub.getClear()); // remove if failure TBD?
|
|
|
setModified();
|
|
|
updateFileAttrs();
|
|
|
- linkSubFile(pos, transaction);
|
|
|
+ linkSubFile(pos);
|
|
|
}
|
|
|
|
|
|
bool doRemoveSubFiles(IDistributedFileTransactionExt *transaction)
|
|
@@ -5732,7 +5850,7 @@ private:
|
|
|
do
|
|
|
{
|
|
|
pos--;
|
|
|
- unlinkSubFile(pos,transaction);
|
|
|
+ unlinkSubFile(pos);
|
|
|
removeItem(pos);
|
|
|
} while (pos);
|
|
|
setModified();
|
|
@@ -5761,7 +5879,7 @@ private:
|
|
|
pos = findSubFile(subfile);
|
|
|
if (pos==NotFound)
|
|
|
return false;
|
|
|
- unlinkSubFile(pos,transaction);
|
|
|
+ unlinkSubFile(pos);
|
|
|
removeItem(pos);
|
|
|
setModified();
|
|
|
updateFileAttrs();
|
|
@@ -7168,25 +7286,27 @@ IDistributedFile *CDistributedFileDirectory::createExternal(const CDfsLogicalFil
|
|
|
return ret;
|
|
|
}
|
|
|
|
|
|
-IDistributedFile *CDistributedFileDirectory::lookup(const char *_logicalname, IUserDescriptor *user, bool writeattr, bool hold, IDistributedFileTransaction *transaction, unsigned timeout)
|
|
|
+IDistributedFile *CDistributedFileDirectory::lookup(const char *_logicalname, IUserDescriptor *user, bool writeattr, bool hold, bool lockSuperOwner, IDistributedFileTransaction *transaction, unsigned timeout)
|
|
|
{
|
|
|
CDfsLogicalFileName logicalname;
|
|
|
logicalname.set(_logicalname);
|
|
|
- return lookup(logicalname, user, writeattr, hold, transaction, timeout);
|
|
|
+ return lookup(logicalname, user, writeattr, hold, lockSuperOwner, transaction, timeout);
|
|
|
}
|
|
|
|
|
|
-IDistributedFile *CDistributedFileDirectory::dolookup(CDfsLogicalFileName &_logicalname, IUserDescriptor *user, bool writeattr, bool hold, IDistributedFileTransaction *transaction, unsigned timeout)
|
|
|
+IDistributedFile *CDistributedFileDirectory::dolookup(CDfsLogicalFileName &_logicalname, IUserDescriptor *user, bool writeattr, bool hold, bool lockSuperOwner, IDistributedFileTransaction *transaction, unsigned timeout)
|
|
|
{
|
|
|
CDfsLogicalFileName *logicalname = &_logicalname;
|
|
|
if (logicalname->isMulti())
|
|
|
// don't bother checking because the sub file creation will
|
|
|
return new CDistributedSuperFile(this,*logicalname,user,transaction); // temp superfile
|
|
|
Owned<IDfsLogicalFileNameIterator> redmatch;
|
|
|
- loop {
|
|
|
+ loop
|
|
|
+ {
|
|
|
checkLogicalName(*logicalname,user,true,writeattr,true,NULL);
|
|
|
if (logicalname->isExternal())
|
|
|
return createExternal(*logicalname); // external always works?
|
|
|
- if (logicalname->isForeign()) {
|
|
|
+ if (logicalname->isForeign())
|
|
|
+ {
|
|
|
IDistributedFile * ret = getFile(logicalname->get(),user,NULL);
|
|
|
if (ret)
|
|
|
return ret;
|
|
@@ -7194,11 +7314,15 @@ IDistributedFile *CDistributedFileDirectory::dolookup(CDfsLogicalFileName &_logi
|
|
|
else {
|
|
|
unsigned start = 0;
|
|
|
loop {
|
|
|
- CFileConnectLock fcl;
|
|
|
- DfsXmlBranchKind bkind;
|
|
|
- if (!fcl.initany("CDistributedFileDirectory::lookup", *logicalname, bkind, false, true, hold, timeout))
|
|
|
+ CFileLock fcl;
|
|
|
+ unsigned mode = RTM_LOCK_READ | RTM_SUB;
|
|
|
+ if (hold) mode |= RTM_LOCK_HOLD;
|
|
|
+ if (!fcl.init(*logicalname, mode, timeout, "CDistributedFileDirectory::lookup"))
|
|
|
break;
|
|
|
- if (bkind == DXB_File)
|
|
|
+ CFileSuperOwnerLock superOwnerLock;
|
|
|
+ if (lockSuperOwner)
|
|
|
+ verifyex(superOwnerLock.init(*logicalname, NULL, defaultTimeout, "CDistributedFileDirectory::dolookup(SuperOwnerLock)"));
|
|
|
+ if (fcl.getKind() == DXB_File)
|
|
|
{
|
|
|
StringBuffer cname;
|
|
|
if (logicalname->getCluster(cname).length())
|
|
@@ -7212,18 +7336,24 @@ IDistributedFile *CDistributedFileDirectory::dolookup(CDfsLogicalFileName &_logi
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
- return new CDistributedFile(this,fcl.detach(),*logicalname,user); // found
|
|
|
+ CDistributedFile *ret = new CDistributedFile(this,fcl.detach(),*logicalname,user); // found
|
|
|
+ ret->setSuperOwnerLock(superOwnerLock.detach());
|
|
|
+ return ret;
|
|
|
}
|
|
|
// now super file
|
|
|
- if (bkind != DXB_SuperFile)
|
|
|
+ if (fcl.getKind() != DXB_SuperFile)
|
|
|
break;
|
|
|
if (start==0)
|
|
|
start = msTick();
|
|
|
unsigned elapsed;
|
|
|
- try {
|
|
|
- return new CDistributedSuperFile(this,fcl.detach(),*logicalname,user,transaction,SDS_SUB_LOCK_TIMEOUT);
|
|
|
+ try
|
|
|
+ {
|
|
|
+ CDistributedSuperFile *ret = new CDistributedSuperFile(this,fcl.detach(),*logicalname,user,transaction,SDS_SUB_LOCK_TIMEOUT);
|
|
|
+ ret->setSuperOwnerLock(superOwnerLock.detach());
|
|
|
+ return ret;
|
|
|
}
|
|
|
- catch (IDFS_Exception *e) {
|
|
|
+ catch (IDFS_Exception *e)
|
|
|
+ {
|
|
|
elapsed = msTick()-start;
|
|
|
if ((e->errorCode()!=DFSERR_LookupConnectionTimout)||(elapsed>((timeout==INFINITE)?SDS_CONNECT_TIMEOUT:timeout)))
|
|
|
throw;
|
|
@@ -7251,16 +7381,16 @@ IDistributedFile *CDistributedFileDirectory::dolookup(CDfsLogicalFileName &_logi
|
|
|
return NULL;
|
|
|
}
|
|
|
|
|
|
-IDistributedFile *CDistributedFileDirectory::lookup(CDfsLogicalFileName &logicalname, IUserDescriptor *user, bool writeattr, bool hold, IDistributedFileTransaction *transaction, unsigned timeout)
|
|
|
+IDistributedFile *CDistributedFileDirectory::lookup(CDfsLogicalFileName &logicalname, IUserDescriptor *user, bool writeattr, bool hold, bool lockSuperOwner, IDistributedFileTransaction *transaction, unsigned timeout)
|
|
|
{
|
|
|
- return dolookup(logicalname, user, writeattr, hold, transaction, timeout);
|
|
|
+ return dolookup(logicalname, user, writeattr, hold, lockSuperOwner, transaction, timeout);
|
|
|
}
|
|
|
|
|
|
IDistributedSuperFile *CDistributedFileDirectory::lookupSuperFile(const char *_logicalname,IUserDescriptor *user,IDistributedFileTransaction *transaction, unsigned timeout)
|
|
|
{
|
|
|
CDfsLogicalFileName logicalname;
|
|
|
logicalname.set(_logicalname);
|
|
|
- IDistributedFile *file = dolookup(logicalname, user, false, false, transaction, timeout);
|
|
|
+ IDistributedFile *file = dolookup(logicalname, user, false, false, false, transaction, timeout);
|
|
|
if (file) {
|
|
|
IDistributedSuperFile *sf = file->querySuperFile();
|
|
|
if (sf)
|
|
@@ -7292,7 +7422,7 @@ bool CDistributedFileDirectory::exists(const char *_logicalname,IUserDescriptor
|
|
|
external = dlfn.isExternal();
|
|
|
foreign = dlfn.isForeign();
|
|
|
if (foreign) {
|
|
|
- Owned<IDistributedFile> file = lookup(_logicalname, user, false, false, NULL, defaultTimeout);
|
|
|
+ Owned<IDistributedFile> file = lookup(_logicalname, user, false, false, false, NULL, defaultTimeout);
|
|
|
if (file.get()==NULL)
|
|
|
return false;
|
|
|
if (file->querySuperFile()) {
|
|
@@ -7327,7 +7457,7 @@ bool CDistributedFileDirectory::exists(const char *_logicalname,IUserDescriptor
|
|
|
|
|
|
bool CDistributedFileDirectory::existsPhysical(const char *_logicalname, IUserDescriptor *user)
|
|
|
{
|
|
|
- Owned<IDistributedFile> file = lookup(_logicalname, user, false, false, NULL, defaultTimeout);
|
|
|
+ Owned<IDistributedFile> file = lookup(_logicalname, user, false, false, false, NULL, defaultTimeout);
|
|
|
if (!file)
|
|
|
return false;
|
|
|
return file->existsPhysicalPartFiles(0);
|
|
@@ -10398,8 +10528,8 @@ DistributedFileCompareResult CDistributedFileDirectory::fileCompare(const char *
|
|
|
StringBuffer msg;
|
|
|
try
|
|
|
{
|
|
|
- Owned<IDistributedFile> file1 = lookup(lfn1, user, false, false, NULL, defaultTimeout);
|
|
|
- Owned<IDistributedFile> file2 = lookup(lfn2, user, false, false, NULL, defaultTimeout);
|
|
|
+ Owned<IDistributedFile> file1 = lookup(lfn1, user, false, false, false, NULL, defaultTimeout);
|
|
|
+ Owned<IDistributedFile> file2 = lookup(lfn2, user, false, false, false, NULL, defaultTimeout);
|
|
|
if (!file1)
|
|
|
{
|
|
|
errstr.appendf("File %s not found",lfn1);
|
|
@@ -10546,7 +10676,7 @@ DistributedFileCompareResult CDistributedFileDirectory::fileCompare(const char *
|
|
|
bool CDistributedFileDirectory::filePhysicalVerify(const char *lfn, IUserDescriptor *user, bool includecrc, StringBuffer &errstr)
|
|
|
{
|
|
|
bool differs = false;
|
|
|
- Owned<IDistributedFile> file = lookup(lfn, user, false, false, NULL, defaultTimeout);
|
|
|
+ Owned<IDistributedFile> file = lookup(lfn, user, false, false, false, NULL, defaultTimeout);
|
|
|
if (!file)
|
|
|
{
|
|
|
errstr.appendf("Could not find file: %s",lfn);
|
|
@@ -10760,7 +10890,7 @@ bool decodeChildGroupName(const char *gname,StringBuffer &parentname, StringBuff
|
|
|
|
|
|
class CLightWeightSuperFileConn: public CInterface, implements ISimpleSuperFileEnquiry
|
|
|
{
|
|
|
- CFileConnectLock lock;
|
|
|
+ CFileLock lock;
|
|
|
bool readonly;
|
|
|
IArrayOf<IRemoteConnection> children;
|
|
|
unsigned defaultTimeout;
|
|
@@ -10864,7 +10994,8 @@ public:
|
|
|
throw MakeStringException(-1,"%s: Invalid superfile name '%s'",title,name);
|
|
|
if (lfn.isMulti()||lfn.isExternal()||lfn.isForeign())
|
|
|
return false;
|
|
|
- if (!lock.init(title, lfn, DXB_SuperFile, !readonly, true, false, timeout))
|
|
|
+ unsigned mode = RTM_SUB | (readonly ? RTM_LOCK_READ : RTM_LOCK_WRITE);
|
|
|
+ if (!lock.init(lfn, DXB_SuperFile, mode, timeout, title))
|
|
|
{
|
|
|
if (!autocreate) // NB not !*autocreate here !
|
|
|
return false;
|
|
@@ -10873,7 +11004,8 @@ public:
|
|
|
root->setPropInt("@numsubfiles",0);
|
|
|
root->setPropTree("Attr",getEmptyAttr());
|
|
|
parent->addEntry(lfn,root,true,false);
|
|
|
- if (!lock.init(title, lfn, DXB_SuperFile, true, true, false, timeout))
|
|
|
+ mode = RTM_SUB | RTM_LOCK_WRITE;
|
|
|
+ if (!lock.init(lfn, DXB_SuperFile, mode, timeout, title))
|
|
|
throw MakeStringException(-1,"%s: Cannot create superfile '%s'",title,name);
|
|
|
if (autocreate)
|
|
|
*autocreate = true;
|
|
@@ -10886,7 +11018,7 @@ public:
|
|
|
|
|
|
void disconnect(bool commit)
|
|
|
{
|
|
|
- if (lock.conn()&&!readonly) {
|
|
|
+ if (lock.queryConnection()&&!readonly) {
|
|
|
if (commit) {
|
|
|
migrateSuperOwnersAttr(lock.queryRoot());
|
|
|
CDateTime dt;
|
|
@@ -10897,10 +11029,10 @@ public:
|
|
|
else {
|
|
|
ForEachItemIn(i,children)
|
|
|
children.item(i).rollback();
|
|
|
- lock.conn()->rollback();
|
|
|
+ lock.queryConnection()->rollback();
|
|
|
}
|
|
|
}
|
|
|
- lock.kill();
|
|
|
+ lock.clear();
|
|
|
children.kill();
|
|
|
}
|
|
|
|
|
@@ -11060,15 +11192,16 @@ ISimpleSuperFileEnquiry * CDistributedFileDirectory::getSimpleSuperFileEnquiry(c
|
|
|
|
|
|
bool CDistributedFileDirectory::getFileSuperOwners(const char *logicalname, StringArray &owners)
|
|
|
{
|
|
|
- CFileConnectLock lock;
|
|
|
+ CFileLock lock;
|
|
|
CDfsLogicalFileName lfn;
|
|
|
if (!lfn.setValidate(logicalname))
|
|
|
throw MakeStringException(-1,"CDistributedFileDirectory::getFileSuperOwners: Invalid file name '%s'",logicalname);
|
|
|
if (lfn.isMulti()||lfn.isExternal()||lfn.isForeign())
|
|
|
return false;
|
|
|
- DfsXmlBranchKind bkind;
|
|
|
- if (!lock.initany("CDistributedFileDirectory::getFileSuperOwners", lfn, bkind, false, false, false, defaultTimeout))
|
|
|
+ if (!lock.init(lfn, RTM_LOCK_READ, defaultTimeout, "CDistributedFileDirectory::getFileSuperOwners"))
|
|
|
return false;
|
|
|
+ CFileSuperOwnerLock superOwnerLock;
|
|
|
+ verifyex(superOwnerLock.init(lfn, NULL, defaultTimeout, "CDistributedFileDirectory::getFileSuperOwners(SuperOwnerLock)"));
|
|
|
Owned<IPropertyTreeIterator> iter = lock.queryRoot()->getElements("SuperOwner");
|
|
|
StringBuffer pname;
|
|
|
ForEach(*iter) {
|
|
@@ -11613,13 +11746,12 @@ IFileDescriptor *CDistributedFileDirectory::createDescriptorFromMetaFile(const C
|
|
|
|
|
|
// Overwrite protection
|
|
|
|
|
|
-bool CDistributedFileDirectory::isProtectedFile(const CDfsLogicalFileName &logicalname, unsigned timeout)
|
|
|
+bool CDistributedFileDirectory::isProtectedFile(const CDfsLogicalFileName &logicalName, unsigned timeout)
|
|
|
{
|
|
|
- DfsXmlBranchKind bkind;
|
|
|
- CFileConnectLock fconnattrlock(true);
|
|
|
- if (!fconnattrlock.initany("CDistributedFileDirectory::isProtectedFile", logicalname, bkind, true, false, false, timeout?timeout:INFINITE))
|
|
|
+ CFileAttrLock attrLock;
|
|
|
+ if (!attrLock.init(logicalName, RTM_LOCK_READ, NULL, timeout?timeout:INFINITE, "CDistributedFileDirectory::isProtectedFile"))
|
|
|
return false; // timeout will raise exception
|
|
|
- Owned<IPropertyTreeIterator> wpiter = fconnattrlock.queryRoot()->getElements("Protect");
|
|
|
+ Owned<IPropertyTreeIterator> wpiter = attrLock.queryRoot()->getElements("Protect");
|
|
|
bool prot = false;
|
|
|
ForEach(*wpiter) {
|
|
|
IPropertyTree &t = wpiter->query();
|
|
@@ -11632,13 +11764,12 @@ bool CDistributedFileDirectory::isProtectedFile(const CDfsLogicalFileName &logic
|
|
|
return prot;
|
|
|
}
|
|
|
|
|
|
-unsigned CDistributedFileDirectory::queryProtectedCount(const CDfsLogicalFileName &logicalname, const char *owner)
|
|
|
+unsigned CDistributedFileDirectory::queryProtectedCount(const CDfsLogicalFileName &logicalName, const char *owner)
|
|
|
{
|
|
|
- DfsXmlBranchKind bkind;
|
|
|
- CFileConnectLock fconnattrlock(true);
|
|
|
- if (!fconnattrlock.initany("CDistributedFileDirectory::isProtectedFile", logicalname, bkind, true, false, false, defaultTimeout))
|
|
|
+ CFileAttrLock attrLock;
|
|
|
+ if (!attrLock.init(logicalName, RTM_LOCK_READ, NULL, defaultTimeout, "CDistributedFileDirectory::queryProtectedCount"))
|
|
|
return 0; // timeout will raise exception
|
|
|
- Owned<IPropertyTreeIterator> wpiter = fconnattrlock.queryRoot()->getElements("Protect");
|
|
|
+ Owned<IPropertyTreeIterator> wpiter = attrLock.queryRoot()->getElements("Protect");
|
|
|
unsigned count = 0;
|
|
|
ForEach(*wpiter) {
|
|
|
IPropertyTree &t = wpiter->query();
|
|
@@ -11649,13 +11780,12 @@ unsigned CDistributedFileDirectory::queryProtectedCount(const CDfsLogicalFileNam
|
|
|
return count;
|
|
|
}
|
|
|
|
|
|
-bool CDistributedFileDirectory::getProtectedInfo(const CDfsLogicalFileName &logicalname, StringArray &names, UnsignedArray &counts)
|
|
|
+bool CDistributedFileDirectory::getProtectedInfo(const CDfsLogicalFileName &logicalName, StringArray &names, UnsignedArray &counts)
|
|
|
{
|
|
|
- DfsXmlBranchKind bkind;
|
|
|
- CFileConnectLock fconnattrlock(true);
|
|
|
- if (!fconnattrlock.initany("CDistributedFileDirectory::isProtectedFile", logicalname, bkind, true, false, false, defaultTimeout))
|
|
|
+ CFileAttrLock attrLock;
|
|
|
+ if (!attrLock.init(logicalName, RTM_LOCK_READ, NULL, defaultTimeout, "CDistributedFileDirectory::getProtectedInfo"))
|
|
|
return false; // timeout will raise exception
|
|
|
- Owned<IPropertyTreeIterator> wpiter = fconnattrlock.queryRoot()->getElements("Protect");
|
|
|
+ Owned<IPropertyTreeIterator> wpiter = attrLock.queryRoot()->getElements("Protect");
|
|
|
bool prot = false;
|
|
|
ForEach(*wpiter) {
|
|
|
IPropertyTree &t = wpiter->query();
|